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 56cdc22f3626..dc4f289ae7f8 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; } @@ -143,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(); @@ -203,7 +202,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 744c3008ca50..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 @@ -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$; @@ -119,8 +118,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); @@ -298,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) { @@ -411,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; } @@ -445,13 +443,7 @@ 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()); - } + taskContext.taskMetrics().incPeakExecutionMemory(getPeakMemoryUsedBytes()); if (stopping) { return Option.apply(null); 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/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 68dc0c6d415f..a6edc1ad3f66 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 @@ -122,9 +122,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/AccumulableParam.scala b/core/src/main/scala/org/apache/spark/AccumulableParam.scala new file mode 100644 index 000000000000..0a97816d2856 --- /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.{BlockId, BlockStatus} + + +/** + * 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 5592b75afb75..9414394cc9fa 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -18,16 +18,15 @@ 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.Map import scala.collection.mutable import scala.ref.WeakReference -import scala.reflect.ClassTag -import org.apache.spark.serializer.JavaSerializer 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`. @@ -37,40 +36,67 @@ 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 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 * @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, +class Accumulable[R, T] private ( + val id: Long, + @transient initialValue: R, param: AccumulableParam[R, T], val name: Option[String], - internal: Boolean) + internal: Boolean, + 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 master - val zero = param.zero(initialValue) // Zero value to be passed to workers + @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 - Accumulators.register(this) + // 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) + } /** * If this [[Accumulable]] is internal. Internal [[Accumulable]]s will be reported to the driver @@ -79,17 +105,28 @@ class Accumulable[R, T] private[spark] ( */ private[spark] def isInternal: Boolean = internal + /** + * 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) + } + /** * 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 @@ -97,7 +134,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 @@ -105,10 +142,10 @@ 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. + * Access the accumulator's current value; only allowed on driver. */ def value: R = { if (!deserialized) { @@ -130,7 +167,7 @@ class Accumulable[R, T] private[spark] ( def localValue: R = value_ /** - * Set the accumulator's value; only allowed on master. + * Set the accumulator's value; only allowed on driver. */ def value_= (newValue: R) { if (!deserialized) { @@ -141,11 +178,15 @@ class Accumulable[R, T] private[spark] ( } /** - * Set the accumulator's value; only allowed on master + * Set the accumulator's value. For internal use only. */ - def setValue(newValue: R) { - this.value = newValue - } + 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 { @@ -153,10 +194,8 @@ class Accumulable[R, T] private[spark] ( 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 - // and are registered in the TaskContext constructor. Other internal accumulators, such SQL - // metrics, still need to register here. + // 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) @@ -166,64 +205,6 @@ class Accumulable[R, T] private[spark] ( 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 @@ -253,14 +234,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) @@ -271,48 +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 - } - - // TODO: Add AccumulatorParams for other types, e.g. lists and strings -} // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right @@ -321,79 +264,63 @@ 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. */ + @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 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 [[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. + * 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 happens 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) - } - - def remove(accId: Long) { - synchronized { - originals.remove(accId) + if (!originals.contains(a.id)) { + originals(a.id) = new WeakReference[Accumulable[_, _]](a) } } - // 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)) { - // 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.") - } - } else { - logWarning(s"Ignoring accumulator update for unknown accumulator id $id") - } - } + /** + * Unregister the [[Accumulable]] with the given ID, if any. + */ + def remove(accId: Long): Unit = synchronized { + originals.remove(accId) } -} - -private[spark] object InternalAccumulator { - 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 + /** + * Return the [[Accumulable]] registered with the given ID, if any. + */ + 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 { + case Some(accum) => accum + case None => + throw new IllegalAccessError(s"Attempted to access garbage collected accumulator $id") + } } } /** - * 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. + * Clear all registered [[Accumulable]]s; for testing only. */ - 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)) - } - internalAccumulators + def clear(): Unit = synchronized { + originals.clear() } + } diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 62629000cfc2..e493d9a3cf9c 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -57,8 +57,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/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 36b536e89c3a..d93d412a37e0 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) { @@ -71,11 +71,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().incUpdatedBlockStatuses(updatedBlocks) new InterruptibleIterator(context, cachedValues) - } finally { loading.synchronized { loading.remove(key) 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/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala new file mode 100644 index 000000000000..cb202a21b90a --- /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.{BlockId, BlockStatus} + + +/** + * 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. + */ + 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/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 7704abc13409..9f49cf1c4c9b 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) } } @@ -138,7 +138,6 @@ abstract class TaskContext extends Serializable { */ def taskAttemptId(): Long - /** ::DeveloperApi:: */ @DeveloperApi def taskMetrics(): TaskMetrics @@ -161,20 +160,4 @@ abstract class TaskContext extends Serializable { */ private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit - /** - * 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. - */ - private[spark] def collectInternalAccumulators(): Map[Long, Any] - - /** - * 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] - - /** - * 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 94ff884b742b..27ca46f73d8c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -17,7 +17,7 @@ package org.apache.spark -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager @@ -32,11 +32,15 @@ private[spark] class TaskContextImpl( override val attemptNumber: Int, override val taskMemoryManager: TaskMemoryManager, @transient private val metricsSystem: MetricsSystem, - internalAccumulators: Seq[Accumulator[Long]], - val taskMetrics: TaskMetrics = TaskMetrics.empty) + initialAccumulators: Seq[Accumulator[_]] = InternalAccumulator.create()) extends TaskContext with Logging { + /** + * Metrics associated with this task. + */ + override val taskMetrics: TaskMetrics = new TaskMetrics(initialAccumulators) + // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] @@ -91,24 +95,8 @@ private[spark] class TaskContextImpl( override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) - @transient private val accumulators = new HashMap[Long, Accumulable[_, _]] - - private[spark] override def registerAccumulator(a: Accumulable[_, _]): Unit = synchronized { - accumulators(a.id) = a - } - - private[spark] override def collectInternalAccumulators(): Map[Long, Any] = synchronized { - accumulators.filter(_._2.isInternal).mapValues(_.localValue).toMap + private[spark] override def registerAccumulator(a: Accumulable[_, _]): Unit = { + taskMetrics.registerAccumulator(a) } - 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 - internalAccumulators.foreach(registerAccumulator) - internalAccumulators.map { a => (a.name.get, a) }.toMap - } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 13241b77bf97..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,7 @@ 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 @@ -115,8 +115,8 @@ case class ExceptionFailure( description: String, stackTrace: Array[StackTraceElement], fullStackTrace: String, - metrics: Option[TaskMetrics], - private val exceptionWrapper: Option[ThrowableSerializationWrapper]) + exceptionWrapper: Option[ThrowableSerializationWrapper], + accumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo]) extends TaskFailedReason { /** @@ -124,13 +124,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, + accumUpdates: Seq[AccumulableInfo], + preserveCause: Boolean) { + this(e.getClass.getName, e.getMessage, e.getStackTrace, Utils.exceptionString(e), + if (preserveCause) Some(new ThrowableSerializationWrapper(e)) else None, accumUpdates) } - private[spark] def this(e: Throwable, metrics: Option[TaskMetrics]) { - this(e, metrics, 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/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 8ba3f5e24189..06b5101b1f56 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 (only available in 2.5), so we should do this 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 9b1418436424..9340b71389de 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._ @@ -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, @@ -249,10 +249,11 @@ 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) + // Note: accumulator updates must be collected after TaskMetrics is updated + val accumUpdates = task.collectAccumulatorUpdates() + val directResult = new DirectTaskResult(valueBytes, accumUpdates) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit @@ -297,21 +298,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.updateAccumulators() - m + // Collect latest accumulator values to report back to the driver + val accumulatorUpdates: Seq[AccumulableInfo] = + if (task != null) { + task.metrics.foreach { m => + m.setExecutorRunTime(System.currentTimeMillis() - taskStart) + m.setJvmGCTime(computeTotalGcTime() - startGCTime) + } + task.collectAccumulatorUpdates(taskFailed = true) + } else { + Seq.empty[AccumulableInfo] } - } + 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, preserveCause = false)) } } execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) @@ -418,33 +423,21 @@ 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) { if (taskRunner.task != null) { taskRunner.task.metrics.foreach { metrics => - metrics.updateShuffleReadMetrics() - metrics.updateInputMetrics() + metrics.mergeShuffleReadMetrics() metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) - metrics.updateAccumulators() - - 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())) } } } - 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/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala new file mode 100644 index 000000000000..fb2f3bb99031 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -0,0 +1,87 @@ +/* + * 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.{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 :: + * A collection of accumulators that represents metrics about reading data from external systems. + */ +@DeveloperApi +class InputMetrics private ( + _bytesRead: Accumulator[Long], + _recordsRead: Accumulator[Long], + _readMethod: Accumulator[String]) + extends Serializable { + + 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), + TaskMetrics.getAccum[String](accumMap, InternalAccumulator.input.READ_METHOD)) + } + + /** + * 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 is [[TaskMetrics.registerInputMetrics]]. + */ + private[executor] def this() { + this(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 + + /** + * 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.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/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala new file mode 100644 index 000000000000..170cc0c49b1c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.{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 :: + * A collection of accumulators that represents metrics about writing data to external systems. + */ +@DeveloperApi +class OutputMetrics private ( + _bytesWritten: Accumulator[Long], + _recordsWritten: Accumulator[Long], + _writeMethod: Accumulator[String]) + extends Serializable { + + 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), + TaskMetrics.getAccum[String](accumMap, InternalAccumulator.output.WRITE_METHOD)) + } + + /** + * Total number of bytes written. + */ + def bytesWritten: Long = _bytesWritten.localValue + + /** + * Total number of records written. + */ + def recordsWritten: Long = _recordsWritten.localValue + + /** + * The source to which this task writes its output. + */ + 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 new file mode 100644 index 000000000000..dc80e7802079 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -0,0 +1,118 @@ +/* + * 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.{Accumulator, InternalAccumulator} +import org.apache.spark.annotation.DeveloperApi + + +/** + * :: DeveloperApi :: + * A collection of accumulators that represent metrics about reading shuffle data. + */ +@DeveloperApi +class ShuffleReadMetrics private ( + _remoteBlocksFetched: Accumulator[Int], + _localBlocksFetched: Accumulator[Int], + _remoteBytesRead: Accumulator[Long], + _localBytesRead: Accumulator[Long], + _fetchWaitTime: Accumulator[Long], + _recordsRead: Accumulator[Long]) + extends Serializable { + + private[executor] def this(accumMap: Map[String, Accumulator[_]]) { + this( + 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), + TaskMetrics.getAccum[Long](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 is [[TaskMetrics.registerTempShuffleReadMetrics]] followed by + * [[TaskMetrics.mergeShuffleReadMetrics]]. + */ + 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: Int = _remoteBlocksFetched.localValue + + /** + * Number of local blocks fetched in this shuffle by this task. + */ + def localBlocksFetched: Int = _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: Int = remoteBlocksFetched + localBlocksFetched + + 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: 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) + 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..efe83854c3d3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -0,0 +1,80 @@ +/* + * 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.{Accumulator, InternalAccumulator} +import org.apache.spark.annotation.DeveloperApi + + +/** + * :: DeveloperApi :: + * A collection of accumulators that represent metrics about writing shuffle data. + */ +@DeveloperApi +class ShuffleWriteMetrics private ( + _bytesWritten: Accumulator[Long], + _recordsWritten: Accumulator[Long], + _writeTime: Accumulator[Long]) + extends Serializable { + + 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), + TaskMetrics.getAccum[Long](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 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 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 = _writeTime.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 42207a955359..87adee909d75 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,421 +17,399 @@ package org.apache.spark.executor -import java.io.{IOException, ObjectInputStream} -import java.util.concurrent.ConcurrentHashMap - +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.DataReadMethod.DataReadMethod +import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} -import org.apache.spark.util.Utils + + +// 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. + * 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. + * + * 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. * - * 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. + * @param initialAccums the initial set of accumulators that this [[TaskMetrics]] depends on. + * 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 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 private[spark](initialAccums: Seq[Accumulator[_]]) extends Serializable { - /** - * Time taken on the executor to deserialize this task - */ - private var _executorDeserializeTime: Long = _ - def executorDeserializeTime: Long = _executorDeserializeTime - private[spark] def setExecutorDeserializeTime(value: Long) = _executorDeserializeTime = value + import InternalAccumulator._ + // Needed for Java tests + def this() { + this(InternalAccumulator.create()) + } /** - * Time the executor spends actually running the task (including fetching shuffle data) + * All accumulators registered with this task. */ - private var _executorRunTime: Long = _ - def executorRunTime: Long = _executorRunTime - private[spark] def setExecutorRunTime(value: Long) = _executorRunTime = value + private val accums = new ArrayBuffer[Accumulable[_, _]] + accums ++= initialAccums /** - * The number of bytes this task transmitted back to the driver as the TaskResult + * A map for quickly accessing the initial set of accumulators by name. */ - private var _resultSize: Long = _ - def resultSize: Long = _resultSize - private[spark] def setResultSize(value: Long) = _resultSize = value + private val initialAccumsMap: Map[String, Accumulator[_]] = { + 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 must be marked as internal") + assert(!map.contains(name), + s"detected duplicate accumulator name '$name' when constructing TaskMetrics") + map(name) = a + } + map.toMap + } + // Each metric is internally represented as an accumulator + 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) + private val _updatedBlockStatuses = + TaskMetrics.getAccum[Seq[(BlockId, BlockStatus)]](initialAccumsMap, UPDATED_BLOCK_STATUSES) /** - * Amount of time the JVM spent in garbage collection while executing this task + * Time taken on the executor to deserialize this task. */ - private var _jvmGCTime: Long = _ - def jvmGCTime: Long = _jvmGCTime - private[spark] def setJvmGCTime(value: Long) = _jvmGCTime = value + def executorDeserializeTime: Long = _executorDeserializeTime.localValue /** - * Amount of time spent serializing the task result + * Time the executor spends actually running the task (including fetching shuffle data). */ - private var _resultSerializationTime: Long = _ - def resultSerializationTime: Long = _resultSerializationTime - private[spark] def setResultSerializationTime(value: Long) = _resultSerializationTime = value + def executorRunTime: Long = _executorRunTime.localValue /** - * The number of in-memory bytes spilled by this task + * The number of bytes this task transmitted back to the driver as the TaskResult. */ - 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 + def resultSize: Long = _resultSize.localValue /** - * The number of on-disk bytes spilled by this task + * Amount of time the JVM spent in garbage collection while executing this task. */ - 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 + def jvmGCTime: Long = _jvmGCTime.localValue /** - * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read - * are stored here. + * Amount of time spent serializing the task result. */ - private var _inputMetrics: Option[InputMetrics] = None - - def inputMetrics: Option[InputMetrics] = _inputMetrics + def resultSerializationTime: Long = _resultSerializationTime.localValue /** - * This should only be used when recreating TaskMetrics, not when updating input metrics in - * executors + * The number of in-memory bytes spilled by this task. */ - private[spark] def setInputMetrics(inputMetrics: Option[InputMetrics]) { - _inputMetrics = inputMetrics - } + def memoryBytesSpilled: Long = _memoryBytesSpilled.localValue /** - * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much - * data was written are stored here. + * The number of on-disk bytes spilled by this task. */ - var outputMetrics: Option[OutputMetrics] = None + def diskBytesSpilled: Long = _diskBytesSpilled.localValue /** - * 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. + * 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 var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None - - def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics + def peakExecutionMemory: Long = _peakExecutionMemory.localValue /** - * This should only be used when recreating TaskMetrics, not when updating read metrics in - * executors. + * Storage statuses of any blocks that have been updated as a result of this task. */ - private[spark] def setShuffleReadMetrics(shuffleReadMetrics: Option[ShuffleReadMetrics]) { - _shuffleReadMetrics = shuffleReadMetrics - } + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue - /** - * ShuffleReadMetrics per dependency for collecting independently while task is in progress. - */ - @transient private lazy val depsShuffleReadMetrics: ArrayBuffer[ShuffleReadMetrics] = - new ArrayBuffer[ShuffleReadMetrics]() + 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)]): Unit = + _updatedBlockStatuses.setValue(v) - /** - * If this task writes to shuffle output, metrics on the written shuffle data will be collected - * here - */ - var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + + /* ============================ * + | OUTPUT METRICS | + * ============================ */ + + private var _outputMetrics: Option[OutputMetrics] = None /** - * Storage statuses of any blocks that have been updated as a result of this task. + * Metrics related to writing data externally (e.g. to a distributed filesystem), + * defined only in tasks with output. */ - var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None + def outputMetrics: Option[OutputMetrics] = _outputMetrics /** - * 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. + * Get or create a new [[OutputMetrics]] associated with this task. */ - private [spark] def createShuffleReadMetricsForDependency(): ShuffleReadMetrics = synchronized { - val readMetrics = new ShuffleReadMetrics() - depsShuffleReadMetrics += readMetrics - readMetrics + def registerOutputMetrics(writeMethod: DataWriteMethod.Value): OutputMetrics = synchronized { + _outputMetrics.getOrElse { + val metrics = new OutputMetrics(initialAccumsMap.toMap) + metrics.setWriteMethod(writeMethod) + _outputMetrics = Some(metrics) + metrics + } } + + /* ========================== * + | INPUT METRICS | + * ========================== */ + + private var _inputMetrics: Option[InputMetrics] = 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). + * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted + * data, defined only in tasks with input. */ - 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) - } - } - } + def inputMetrics: Option[InputMetrics] = _inputMetrics /** - * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. + * Get or create a new [[InputMetrics]] associated with this task. */ - 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) + private[spark] def registerInputMetrics(readMethod: DataReadMethod.Value): InputMetrics = { + synchronized { + val metrics = _inputMetrics.getOrElse { + val metrics = new InputMetrics(initialAccumsMap.toMap) + metrics.setReadMethod(readMethod) + _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 { + val m = new InputMetrics + m.setReadMethod(readMethod) + m } - _shuffleReadMetrics = Some(merged) } } - private[spark] def updateInputMetrics(): Unit = synchronized { - 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 + /* =================================== * + | SHUFFLE WRITE METRICS | + * =================================== */ - private[spark] def updateAccumulators(): Unit = synchronized { - _accumulatorUpdates = _accumulatorsUpdater() - } + private var _shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None /** - * Return the latest updates of accumulators in this task. + * Metrics related to shuffle write, defined only in shuffle map stages. */ - def accumulatorUpdates(): Map[Long, Any] = _accumulatorUpdates - - private[spark] def setAccumulatorsUpdater(accumulatorsUpdater: () => Map[Long, Any]): Unit = { - _accumulatorsUpdater = accumulatorsUpdater - } -} - -private[spark] object TaskMetrics { - private val hostNameCache = new ConcurrentHashMap[String, String]() - - def empty: TaskMetrics = new TaskMetrics + def shuffleWriteMetrics: Option[ShuffleWriteMetrics] = _shuffleWriteMetrics - def getCachedHostName(host: String): String = { - val canonicalHost = hostNameCache.putIfAbsent(host, host) - if (canonicalHost != null) canonicalHost else host + /** + * Get or create a new [[ShuffleWriteMetrics]] associated with this task. + */ + def registerShuffleWriteMetrics(): ShuffleWriteMetrics = synchronized { + _shuffleWriteMetrics.getOrElse { + val metrics = new ShuffleWriteMetrics(initialAccumsMap.toMap) + _shuffleWriteMetrics = Some(metrics) + metrics + } } -} -/** - * :: 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 -} + /* ================================== * + | SHUFFLE READ METRICS | + * ================================== */ -/** - * :: DeveloperApi :: - * Metrics about reading input data. - */ -@DeveloperApi -case class InputMetrics(readMethod: DataReadMethod.Value) { + private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None /** - * This is volatile so that it is visible to the updater thread. + * Metrics related to shuffle read aggregated across all shuffle dependencies. + * This is defined only if there are shuffle dependencies in this task. */ - @volatile @transient var bytesReadCallback: Option[() => Long] = None + def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics /** - * Total bytes read. + * 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. */ - private var _bytesRead: Long = _ - def bytesRead: Long = _bytesRead - def incBytesRead(bytes: Long): Unit = _bytesRead += bytes + @transient private lazy val tempShuffleReadMetrics = new ArrayBuffer[ShuffleReadMetrics] /** - * Total records read. + * 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. Otherwise, all temporary data collected will + * be lost. */ - private var _recordsRead: Long = _ - def recordsRead: Long = _recordsRead - def incRecordsRead(records: Long): Unit = _recordsRead += records + private[spark] def registerTempShuffleReadMetrics(): ShuffleReadMetrics = synchronized { + val readMetrics = new ShuffleReadMetrics + tempShuffleReadMetrics += readMetrics + readMetrics + } /** - * Invoke the bytesReadCallback and mutate bytesRead. + * 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 updateBytesRead() { - bytesReadCallback.foreach { c => - _bytesRead = c() + private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { + 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) } } - /** - * 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 - } -} - -/** - * :: DeveloperApi :: - * Metrics about writing output data. - */ -@DeveloperApi -case class OutputMetrics(writeMethod: DataWriteMethod.Value) { - /** - * Total bytes written - */ - private var _bytesWritten: Long = _ - def bytesWritten: Long = _bytesWritten - private[spark] def setBytesWritten(value : Long): Unit = _bytesWritten = value - /** - * Total records written - */ - private var _recordsWritten: Long = 0L - def recordsWritten: Long = _recordsWritten - private[spark] def setRecordsWritten(value: Long): Unit = _recordsWritten = value -} + /* ========================== * + | OTHER THINGS | + * ========================== */ -/** - * :: DeveloperApi :: - * Metrics pertaining to shuffle data read in a given task. - */ -@DeveloperApi -class ShuffleReadMetrics extends Serializable { /** - * Number of remote blocks fetched in this shuffle by this task + * Register an accumulator with this task so we can access its value in [[accumulatorUpdates]]. */ - private var _remoteBlocksFetched: Int = _ - def remoteBlocksFetched: Int = _remoteBlocksFetched - private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value - private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value + private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit = { + accums += a + } /** - * Number of local blocks fetched in this shuffle by this task + * 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. */ - private var _localBlocksFetched: Int = _ - def localBlocksFetched: Int = _localBlocksFetched - private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value - private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value + private[spark] def getAccum(name: String): Accumulator[Long] = { + TaskMetrics.getAccum[Long](initialAccumsMap, name) + } /** - * 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. + * Return a map from accumulator ID to the accumulator's latest value in this task. */ - private var _fetchWaitTime: Long = _ - def fetchWaitTime: Long = _fetchWaitTime - private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value - private[spark] def decFetchWaitTime(value: Long) = _fetchWaitTime -= value + def accumulatorUpdates(): Seq[AccumulableInfo] = accums.map { a => + new AccumulableInfo( + a.id, a.name.orNull, Some(a.localValue), None, a.isInternal, a.countFailedValues) + } - /** - * 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[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value + // 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. + { + var (hasShuffleRead, hasShuffleWrite, hasInput, hasOutput) = (false, false, false, false) + initialAccums + .filter { a => a.localValue != a.zero } + .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 + 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)) } + } - /** - * 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 +} - /** - * 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: Int = _remoteBlocksFetched + _localBlocksFetched +private[spark] object TaskMetrics extends Logging { - /** - * Total number of records read from the shuffle by this task - */ - private var _recordsRead: Long = _ - def recordsRead: Long = _recordsRead - private[spark] def incRecordsRead(value: Long) = _recordsRead += value - private[spark] def decRecordsRead(value: Long) = _recordsRead -= value -} + def empty: TaskMetrics = new TaskMetrics -/** - * :: DeveloperApi :: - * Metrics pertaining to shuffle data written in a given task. - */ -@DeveloperApi -class ShuffleWriteMetrics extends Serializable { /** - * Number of bytes written for the shuffle by this task + * Get an accumulator from the given map by name, assuming it exists. */ - @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 getAccum[T]( + 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 + accum.asInstanceOf[Accumulator[T]] + } catch { + case e: ClassCastException => + throw new SparkException(s"accumulator $name was of unexpected type", e) + } + } /** - * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds - */ - @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 + * 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. + * + * 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 + // 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 (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 + 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.") + } + acc + } + val metrics = new TaskMetrics(initialAccums) + otherAccums.foreach(metrics.registerAccumulator) + metrics + } - /** - * Total number of records written to the shuffle by this task - */ - @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 - private[spark] def setShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten = value } 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..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,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().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/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 3587e7eb1afa..d9b0824b38ec 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -153,8 +153,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/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index a7a6e0b8a94f..da9847d953cc 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 { @@ -222,14 +222,17 @@ class HadoopRDD[K, V]( // 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 +255,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 +278,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 7a1197830443..349a398fad88 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -129,19 +129,21 @@ 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) // 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 { @@ -183,6 +185,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) } @@ -202,14 +207,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/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 16a856f594e9..3d164d56adca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -1092,8 +1092,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 @@ -1177,8 +1177,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 @@ -1204,17 +1204,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/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index 146cfb9ba803..8008dfcf2516 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -19,47 +19,32 @@ 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. + * + * 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], + 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, false, 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 6b01a10fc136..e1ab434e475e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -208,11 +208,9 @@ class DAGScheduler( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics): Unit = { - eventProcessLoop.post( - CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) + accumUpdates: Seq[AccumulableInfo], + taskInfo: TaskInfo): Unit = { + eventProcessLoop.post(CompletionEvent(task, reason, result, accumUpdates, taskInfo)) } /** @@ -222,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")) } @@ -1033,7 +1032,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) @@ -1074,39 +1072,43 @@ class DAGScheduler( } } - /** Merge updates from a task to our local accumulator values */ + /** + * 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 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 { - case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] - case None => throw new NullPointerException("Non-existent reference to Accumulator") - } - - // 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) - event.taskInfo.accumulables += - new AccumulableInfo(id, name, Some(s"$partialValue"), value, acc.isInternal) - } + try { + event.accumUpdates.foreach { ainfo => + assert(ainfo.update.isDefined, "accumulator from task should have a partial value") + val id = ainfo.id + val partialValue = ainfo.update.get + // 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 => + 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, acc.countFailedValues) + event.taskInfo.accumulables += new AccumulableInfo( + id, name, Some(partialValue), Some(acc.value), acc.isInternal, acc.countFailedValues) } - } 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) } + } catch { + case NonFatal(e) => + logError(s"Failed to update accumulators for task ${task.partitionId}", e) } } @@ -1116,22 +1118,32 @@ class DAGScheduler( */ private[scheduler] def handleTaskCompletion(event: CompletionEvent) { val task = event.task + val taskId = task.partitionId val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) outputCommitCoordinator.taskCompleted( stageId, - task.partitionId, + taskId, 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, event.taskMetrics)) - } + // Reconstruct task metrics. Note: this may be null if the task has failed. + val taskMetrics: TaskMetrics = + 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 + } + + 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. @@ -1141,8 +1153,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, event.taskMetrics)) stage.pendingPartitions -= task.partitionId task match { case rt: ResultTask[_, _] => @@ -1291,7 +1301,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. + updateAccumulators(event) case TaskResultLost => // Do nothing here; the TaskScheduler handles these failures and resubmits the task. @@ -1637,7 +1648,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..6b6bccf0b39f 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 @@ -73,9 +72,8 @@ private[scheduler] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) + accumUpdates: Seq[AccumulableInfo], + 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/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 6590cf6ffd24..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, @@ -45,8 +49,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..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, @@ -44,8 +48,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/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 3130a65240a9..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 @@ -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/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 fca57928eca1..4a06b729537c 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 @@ -41,19 +41,17 @@ 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, val stageAttemptId: Int, val partitionId: Int, - 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 - * local value. - */ - type AccumulatorUpdates = Map[Long, Any] + val initialAccumulators: Seq[Accumulator[_]]) extends Serializable { /** * Called by [[Executor]] to run this task. @@ -63,10 +61,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, AccumulatorUpdates) = { + taskAttemptId: Long, + attemptNumber: Int, + metricsSystem: MetricsSystem): T = { context = new TaskContextImpl( stageId, partitionId, @@ -74,16 +71,14 @@ private[spark] abstract class Task[T]( attemptNumber, taskMemoryManager, metricsSystem, - internalAccumulators) + initialAccumulators) TaskContext.setTaskContext(context) - context.taskMetrics.setHostname(Utils.localHostName()) - context.taskMetrics.setAccumulatorsUpdater(context.collectInternalAccumulators) taskThread = Thread.currentThread() if (_killed) { kill(interruptThread = false) } try { - (runTask(context), context.collectAccumulators()) + runTask(context) } finally { context.markTaskCompleted() try { @@ -130,6 +125,18 @@ private[spark] abstract class Task[T]( protected var _executorDeserializeTime: Long = 0 + /** + * 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): Seq[AccumulableInfo] = { + if (context != null) { + context.taskMetrics.accumulatorUpdates().filter { a => !taskFailed || a.countFailedValues } + } else { + Seq.empty[AccumulableInfo] + } + } + /** * 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..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,11 +20,9 @@ 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.executor.TaskMetrics import org.apache.spark.storage.BlockId import org.apache.spark.util.Utils @@ -36,31 +34,24 @@ 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: Seq[AccumulableInfo]) 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) - for ((key, value) <- accumUpdates) { - out.writeLong(key) - out.writeObject(value) - } - out.writeObject(metrics) + accumUpdates.foreach(out.writeObject) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - val blen = in.readInt() val byteVal = new Array[Byte](blen) in.readFully(byteVal) @@ -70,13 +61,12 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long 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 } - 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..65ac5e514574 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): Unit = { 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) } - result.metrics.setResultSize(size) + // Set the task result size in the accumulator updates received from the executors. + // 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, + "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 => 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/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index aa39b59d8cce..a74c7f069374 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} @@ -621,8 +620,7 @@ private[spark] class TaskSetManager( // "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( @@ -653,8 +651,7 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - var taskMetrics : TaskMetrics = 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 { @@ -669,7 +666,8 @@ private[spark] class TaskSetManager( None case ef: ExceptionFailure => - taskMetrics = ef.metrics.orNull + // ExceptionFailure's might have accumulator updates + 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" @@ -721,7 +719,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 +791,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, null) + sched.dagScheduler.taskEnded( + tasks(index), Resubmitted, null, Seq.empty[AccumulableInfo], info) } } } 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..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) @@ -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/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 294e16cde193..2970968f0bd4 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/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..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,8 +45,7 @@ private[spark] class SortShuffleWriter[K, V, C]( private var mapStatus: MapStatus = null - private val writeMetrics = new ShuffleWriteMetrics() - context.taskMetrics.shuffleWriteMetrics = Some(writeMetrics) + 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 = { @@ -93,8 +92,7 @@ private[spark] class SortShuffleWriter[K, V, C]( if (sorter != null) { val startTime = System.nanoTime() sorter.stop() - context.taskMetrics.shuffleWriteMetrics.foreach( - _.incShuffleWriteTime(System.nanoTime - startTime)) + writeMetrics.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 341ae782362a..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 @@ -214,9 +214,9 @@ private[v1] object AllStagesResource { raw.shuffleWriteMetrics } def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( - writeBytes = submetricQuantiles(_.shuffleBytesWritten), - writeRecords = submetricQuantiles(_.shuffleRecordsWritten), - writeTime = submetricQuantiles(_.shuffleWriteTime) + writeBytes = submetricQuantiles(_.bytesWritten), + writeRecords = submetricQuantiles(_.recordsWritten), + writeTime = submetricQuantiles(_.writeTime) ) }.metricOption @@ -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 = { @@ -283,9 +287,9 @@ private[v1] object AllStagesResource { def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { new ShuffleWriteMetrics( - bytesWritten = internal.shuffleBytesWritten, - writeTime = internal.shuffleWriteTime, - recordsWritten = internal.shuffleRecordsWritten + bytesWritten = internal.bytesWritten, + writeTime = internal.writeTime, + 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 e36a367323b2..c34d49c0d906 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/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 037bec1d9c33..c6065df64ae0 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/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index ec711480ebf3..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.getOrElse(Seq[(BlockId, BlockStatus)]()) + val updatedBlocks = metrics.updatedBlockStatuses if (updatedBlocks.length > 0) { updateStorageStatus(info.executorId, updatedBlocks) } 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 2d955a66601e..160d7a4dff2d 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..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)) @@ -418,7 +425,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { * aggregate metrics by calculating deltas between the currently recorded metrics and the new * metrics. */ - def updateAggregateMetrics( + private def updateAggregateMetrics( stageData: StageUIData, execId: String, taskMetrics: TaskMetrics, @@ -426,14 +433,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 @@ -489,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/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 6d4066a870cd..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 @@ -404,12 +404,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 = { @@ -500,11 +496,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 +: @@ -619,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) @@ -891,15 +887,10 @@ private[ui] class TaskDataSource( val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = getGettingResultTime(info, currentTime) - val (taskInternalAccumulables, 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 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) val inputSortable = maybeInput.map(_.bytesRead).getOrElse(0L) @@ -930,13 +921,13 @@ 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 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/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 2d9b885c684b..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.isDefined) { - updateRDDInfo(metrics.updatedBlocks.get) + 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 a6460bc8b820..f3c5e33e9c7a 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._ @@ -233,14 +234,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)) }) } @@ -265,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 = { @@ -284,30 +285,77 @@ private[spark] object JsonProtocol { } def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { + val name = accumulableInfo.name ("ID" -> accumulableInfo.id) ~ - ("Name" -> accumulableInfo.name) ~ - ("Update" -> accumulableInfo.update.map(new JString(_)).getOrElse(JNothing)) ~ - ("Value" -> accumulableInfo.value) ~ - ("Internal" -> accumulableInfo.internal) + ("Name" -> name) ~ + ("Update" -> accumulableInfo.update.map { v => accumValueToJson(name, v) }) ~ + ("Value" -> accumulableInfo.value.map { v => accumValueToJson(name, v) }) ~ + ("Internal" -> accumulableInfo.internal) ~ + ("Count Failed Values" -> accumulableInfo.countFailedValues) + } + + /** + * 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 = - 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.bytesWritten) ~ + ("Shuffle Write Time" -> wm.writeTime) ~ + ("Shuffle Records Written" -> wm.recordsWritten) + }.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) - val updatedBlocks = - taskMetrics.updatedBlocks.map { blocks => - JArray(blocks.toList.map { case (id, status) => - ("Block ID" -> id.toString) ~ - ("Status" -> blockStatusToJson(status)) - }) + taskMetrics.outputMetrics.map { om => + ("Data Write Method" -> om.writeMethod.toString) ~ + ("Bytes Written" -> om.bytesWritten) ~ + ("Records Written" -> om.recordsWritten) }.getOrElse(JNothing) - ("Host Name" -> taskMetrics.hostname) ~ + val updatedBlocks = + JArray(taskMetrics.updatedBlockStatuses.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) ~ @@ -322,33 +370,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 { @@ -362,12 +383,12 @@ private[spark] object JsonProtocol { ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) - val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) + val accumUpdates = JArray(exceptionFailure.accumUpdates.map(accumulableInfoToJson).toList) ("Class Name" -> exceptionFailure.className) ~ ("Description" -> exceptionFailure.description) ~ ("Stack Trace" -> stackTrace) ~ ("Full Stack Trace" -> exceptionFailure.fullStackTrace) ~ - ("Metrics" -> metrics) + ("Accumulator Updates" -> accumUpdates) case taskCommitDenied: TaskCommitDenied => ("Job ID" -> taskCommitDenied.jobID) ~ ("Partition ID" -> taskCommitDenied.partitionID) ~ @@ -628,15 +649,16 @@ 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 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] 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(execId, accumUpdates) } /** --------------------------------------------------------------------- * @@ -657,7 +679,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]() } @@ -685,7 +707,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]() } @@ -701,10 +723,41 @@ 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 = 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) - AccumulableInfo(id, name, update, value, internal) + val countFailedValues = (json \ "Count Failed Values").extractOpt[Boolean].getOrElse(false) + new AccumulableInfo(id, name, update, value, internal, countFailedValues) + } + + /** + * 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 = { @@ -712,7 +765,6 @@ private[spark] object JsonProtocol { return TaskMetrics.empty } val metrics = new TaskMetrics - metrics.setHostname((json \ "Host Name").extract[String]) metrics.setExecutorDeserializeTime((json \ "Executor Deserialize Time").extract[Long]) metrics.setExecutorRunTime((json \ "Executor Run Time").extract[Long]) metrics.setResultSize((json \ "Result Size").extract[Long]) @@ -720,58 +772,54 @@ 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)) - metrics.shuffleWriteMetrics = - Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - 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 => - val id = BlockId((block \ "Block ID").extract[String]) - val status = blockStatusFromJson(block \ "Status") - (id, status) - } - } - metrics - } - 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 - } + // Shuffle read metrics + Utils.jsonOption(json \ "Shuffle Read Metrics").foreach { readJson => + val readMetrics = metrics.registerTempShuffleReadMetrics() + 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]) + readMetrics.setRecordsRead((readJson \ "Total Records Read").extractOpt[Long].getOrElse(0L)) + metrics.mergeShuffleReadMetrics() + } - 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.setShuffleRecordsWritten((json \ "Shuffle Records Written") - .extractOpt[Long].getOrElse(0)) - metrics - } + // 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.incBytesWritten((writeJson \ "Shuffle Bytes Written").extract[Long]) + writeMetrics.incRecordsWritten((writeJson \ "Shuffle Records Written") + .extractOpt[Long].getOrElse(0L)) + writeMetrics.incWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) + } - 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 - } + // 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").extractOpt[Long].getOrElse(0L)) + } + + // Input metrics + Utils.jsonOption(json \ "Input Metrics").foreach { inJson => + val readMethod = DataReadMethod.withName((inJson \ "Data Read Method").extract[String]) + val inputMetrics = metrics.registerInputMetrics(readMethod) + inputMetrics.setBytesRead((inJson \ "Bytes Read").extract[Long]) + inputMetrics.incRecordsRead((inJson \ "Records Read").extractOpt[Long].getOrElse(0L)) + } + + // Updated blocks + Utils.jsonOption(json \ "Updated Blocks").foreach { blocksJson => + metrics.setUpdatedBlockStatuses(blocksJson.extract[List[JValue]].map { blockJson => + val id = BlockId((blockJson \ "Block ID").extract[String]) + val status = blockStatusFromJson(blockJson \ "Status") + (id, status) + }) + } - 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 } @@ -801,10 +849,14 @@ 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 metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) - ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics, None) + val fullStackTrace = (json \ "Full Stack Trace").extractOpt[String].orNull + // 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 case `taskKilled` => TaskKilled case `taskCommitDenied` => 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 4a44481cf4e1..0518903ac7d0 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 = new ShuffleWriteMetrics var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) var objectsWritten = 0 @@ -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 } @@ -208,7 +208,7 @@ class ExternalAppendOnlyMap[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() - curWriteMetrics = new ShuffleWriteMetrics() + 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 63ba954a7fa7..5afd6d6e22c6 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 } @@ -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) } @@ -680,8 +682,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/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 5fe64bde3604..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); } @@ -279,8 +277,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 +309,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 +352,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 +414,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 +435,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/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 5b84acf40be4..a818b93545d2 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,193 +159,130 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(!Accumulators.originals.get(accId).isDefined) } - test("internal accumulators in TaskContext") { + test("get accum") { 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_ACCUMULATOR)) - val testAccum = internalMetricsToAccums(TEST_ACCUMULATOR) - assert(collectedInternalAccums.size === internalMetricsToAccums.size) - assert(collectedInternalAccums.size === collectedAccums.size) - assert(collectedInternalAccums.contains(testAccum.id)) - assert(collectedAccums.contains(testAccum.id)) - } + // 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) - 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().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 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 = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) - 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) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - taskAccum.value.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) + // 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) } - rdd.count() + + // 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("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().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 - iter - } - .reduceByKey { case (x, y) => x + y } - .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 10 - iter - } - .repartition(numPartitions * 2) - .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 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) = - (findAccumulableInfo(stageInfos(0).accumulables.values, TEST_ACCUMULATOR), - findAccumulableInfo(stageInfos(1).accumulables.values, TEST_ACCUMULATOR), - findAccumulableInfo(stageInfos(2).accumulables.values, TEST_ACCUMULATOR)) - assert(firstStageAccum.value.toLong === numPartitions) - assert(secondStageAccum.value.toLong === numPartitions * 10) - assert(thirdStageAccum.value.toLong === numPartitions * 2 * 100) - } - rdd.count() + 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("internal accumulators in fully resubmitted stages") { - testInternalAccumulatorsWithFailedTasks((i: Int) => true) // fail all tasks + 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("internal accumulators in partially resubmitted stages") { - testInternalAccumulatorsWithFailedTasks((i: Int) => i % 2 == 0) // fail a subset + 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)) } - /** - * 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) - } + 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 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.internalMetricsToAccumulators(TEST_ACCUMULATOR) += 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 = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) - // 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) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - Some(taskAccum.value.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("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)) } } 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) - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { jobId => - 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)) - } 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) - .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.get.toString.toLong > 0 + } + if (!isSet) { + throw new TestFailedException(s"peak execution memory accumulator not set in '$testName'", 0) + } } } @@ -367,6 +304,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) } } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 30aa94c8a597..1645735c36c9 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,6 +86,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.updatedBlockStatuses.size === 2) } } 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/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/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala new file mode 100644 index 000000000000..9f067d15c28f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -0,0 +1,280 @@ +/* + * 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.scheduler.AccumulableInfo +import org.apache.spark.storage.{BlockId, BlockStatus} + + +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.name === Some(EXECUTOR_RUN_TIME)) + assert(updatedBlockStatuses.name === Some(UPDATED_BLOCK_STATUSES)) + assert(shuffleRemoteBlocksRead.name === Some(shuffleRead.REMOTE_BLOCKS_FETCHED)) + assert(inputReadMethod.name === Some(input.READ_METHOD)) + assert(executorRunTime.value.isInstanceOf[Long]) + assert(updatedBlockStatuses.value.isInstanceOf[Seq[_]]) + // We cannot assert the type of the value directly since the type parameter is erased. + // Instead, try casting a `Seq` of expected type and see if it fails in run time. + updatedBlockStatuses.setValueAny(Seq.empty[(BlockId, BlockStatus)]) + assert(shuffleRemoteBlocksRead.value.isInstanceOf[Int]) + assert(inputReadMethod.value.isInstanceOf[String]) + } + + 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() + } + +} diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index c45d81459e8e..6ffa1c8ac140 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/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/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 8275fd87764c..885f424c2340 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,464 @@ 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.updateShuffleReadMetrics() - assert(taskMetrics.shuffleReadMetrics.isEmpty) + import AccumulatorParam._ + import InternalAccumulator._ + import StorageLevel._ + import TaskMetricsSuite._ + + 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 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) + tm.registerTempShuffleReadMetrics() + tm.mergeShuffleReadMetrics() + assert(tm.shuffleReadMetrics.isDefined) + val sr = tm.shuffleReadMetrics.get + // initial values + 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) + 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 + 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 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) + tm.registerShuffleWriteMetrics() + assert(tm.shuffleWriteMetrics.isDefined) + val sw = tm.shuffleWriteMetrics.get + // initial values + 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 + 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 + 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 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 + assert(tm.inputMetrics.isEmpty) + tm.registerInputMetrics(DataReadMethod.Memory) + assert(tm.inputMetrics.isDefined) + val in = tm.inputMetrics.get + // initial values + 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) + in.incRecordsRead(1L) + in.incRecordsRead(2L) + in.setReadMethod(DataReadMethod.Disk) + // assert new values exist + 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 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 + assert(tm.outputMetrics.isEmpty) + tm.registerOutputMetrics(DataWriteMethod.Hadoop) + assert(tm.outputMetrics.isDefined) + val out = tm.outputMetrics.get + // initial values + assertValEquals(_.bytesWritten, BYTES_WRITTEN, 0L) + assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) + assertValEquals(_.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 + 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 + assertValEquals(_.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) + } +} + + +// 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`: + * (1) TaskMetrics value + * (2) TaskMetrics accumulator update value + * (3) Original accumulator value + */ + 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. + */ + 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. + */ + 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/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala index 0e60cc8e7787..2b5e4b80e96a 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -31,7 +31,6 @@ object MemoryTestingUtils { taskAttemptId = 0, attemptNumber = 0, taskMemoryManager = taskMemoryManager, - metricsSystem = env.metricsSystem, - internalAccumulators = Seq.empty) + metricsSystem = env.metricsSystem) } } 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 aaf62e0f9106..e5a448298a62 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/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 370a284d2950..5c3ee56c9b72 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -23,12 +23,10 @@ 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._ 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} @@ -96,8 +94,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. */ @@ -111,8 +108,10 @@ class DAGSchedulerSuite 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) @@ -189,7 +188,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 +211,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 +253,24 @@ 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, + Seq(new AccumulableInfo( + accumId, "", Some(1), None, internal = false, countFailedValues = false)))) } } } @@ -338,9 +347,12 @@ class DAGSchedulerSuite } 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"), internal = true, countFailedValues = false) + val accInfo2 = new AccumulableInfo( + 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) assert(accInfo2 === accInfo3) assert(accInfo2.hashCode() === accInfo3.hashCode()) @@ -464,7 +476,7 @@ class DAGSchedulerSuite 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 @@ -829,23 +841,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 +888,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 +903,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 +952,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 +968,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 +1001,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 +1122,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 +1134,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 +1150,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 +1209,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( @@ -1606,6 +1573,25 @@ class DAGSchedulerSuite 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("reduce tasks should be placed locally with map output") { // Create an shuffleMapRdd with 1 partition val shuffleMapRdd = new MyRDD(sc, 1, Nil) @@ -1884,8 +1870,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 +1880,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 +1943,23 @@ class DAGSchedulerSuite info } + private def makeCompletionEvent( + task: Task[_], + reason: TaskEndReason, + result: Any, + extraAccumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo], + taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { + 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) + } + } 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/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/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index dc15f5932d6f..c87158d89f3f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -269,7 +269,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/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index e5ec44a9f3b6..784bcc1e0264 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -22,6 +22,8 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark._ +import org.apache.spark.executor.TaskMetricsSuite +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 @@ -58,7 +60,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) } @@ -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 { 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..6da2677be033 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) @@ -38,9 +37,8 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { + accumUpdates: Seq[AccumulableInfo], + taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } @@ -167,14 +165,17 @@ 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 => + new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal, a.countFailedValues) + } // 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)) assert(sched.endedTasks(0) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -184,10 +185,14 @@ 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[Seq[AccumulableInfo]] = + taskSet.tasks.map { _.initialAccumulators.map { a => + new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal, a.countFailedValues) + }} // 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 +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)) - manager.handleSuccessfulTask(1, createTaskResult(1)) + 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)) + manager.handleSuccessfulTask(2, createTaskResult(2, accumUpdatesByTask(2))) assert(sched.endedTasks(2) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -620,7 +625,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 +791,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: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo]): 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/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index ef6ce04e3ff2..fdacd8c9f590 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) } @@ -169,8 +169,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(writer.getPartitionLengths.filter(_ == 0L).size === 4) // should be 4 zero length files 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 5d36617cfc44..078cffc0cb81 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -44,24 +44,24 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { test("verify write metrics") { val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() + val writeMetrics = new ShuffleWriteMetrics val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) 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,29 +109,29 @@ 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") { val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() + val writeMetrics = new ShuffleWriteMetrics val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) for (i <- 1 to 1000) { writer.write(i, i) } writer.commitAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten - val writeTime = writeMetrics.shuffleWriteTime - assert(writeMetrics.shuffleRecordsWritten === 1000) + val bytesWritten = writeMetrics.bytesWritten + val writeTime = writeMetrics.writeTime + assert(writeMetrics.recordsWritten === 1000) writer.commitAndClose() - assert(writeMetrics.shuffleRecordsWritten === 1000) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) - assert(writeMetrics.shuffleWriteTime === writeTime) + assert(writeMetrics.recordsWritten === 1000) + assert(writeMetrics.bytesWritten === bytesWritten) + assert(writeMetrics.writeTime === writeTime) } test("revertPartialWritesAndClose() should be idempotent") { @@ -143,13 +143,13 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { writer.write(i, i) } writer.revertPartialWritesAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten - val writeTime = writeMetrics.shuffleWriteTime - assert(writeMetrics.shuffleRecordsWritten === 0) + val bytesWritten = writeMetrics.bytesWritten + val writeTime = writeMetrics.writeTime + assert(writeMetrics.recordsWritten === 0) writer.revertPartialWritesAndClose() - assert(writeMetrics.shuffleRecordsWritten === 0) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) - assert(writeMetrics.shuffleWriteTime === writeTime) + assert(writeMetrics.recordsWritten === 0) + assert(writeMetrics.bytesWritten === bytesWritten) + 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/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 355d80d06898..9de434166bba 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)) val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L)) val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L)) - taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) - taskMetrics2.updatedBlocks = Some(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)) val droppedBlock2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.NONE, 0L, 0L)) val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L)) - taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) - taskMetrics2.updatedBlocks = Some(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)) val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L)) val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L)) - taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) - taskMetrics2.updatedBlocks = Some(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/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 86699e7f5695..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), (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) 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..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 @@ -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) @@ -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")), @@ -269,23 +269,22 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val execId = "exe-1" def makeTaskMetrics(base: Int): TaskMetrics = { - val taskMetrics = new TaskMetrics() - val shuffleReadMetrics = new ShuffleReadMetrics() - val shuffleWriteMetrics = new ShuffleWriteMetrics() - taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) - taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) + 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) + val outputMetrics = taskMetrics.registerOutputMetrics(DataWriteMethod.Hadoop) shuffleReadMetrics.incRemoteBytesRead(base + 1) shuffleReadMetrics.incLocalBytesRead(base + 9) shuffleReadMetrics.incRemoteBlocksFetched(base + 2) - shuffleWriteMetrics.incShuffleBytesWritten(base + 3) + taskMetrics.mergeShuffleReadMetrics() + shuffleWriteMetrics.incBytesWritten(base + 3) 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 = new OutputMetrics(DataWriteMethod.Hadoop) - taskMetrics.outputMetrics = Some(outputMetrics) + inputMetrics.setBytesRead(base + 7) outputMetrics.setBytesWritten(base + 8) taskMetrics } @@ -303,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/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index 5ac922c2172c..d1dbf7c1558b 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.setUpdatedBlockStatuses(Seq( (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L)), (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L)), (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L)) @@ -146,7 +146,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { // Task end with a few dropped blocks val metrics2 = new TaskMetrics - metrics2.updatedBlocks = Some(Seq( + metrics2.setUpdatedBlockStatuses(Seq( (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L)), (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L)), (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L)), // doesn't actually exist @@ -173,8 +173,8 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { val taskMetrics1 = new TaskMetrics val block0 = (RDDBlockId(0, 1), BlockStatus(memOnly, 100L, 0L)) val block1 = (RDDBlockId(1, 1), BlockStatus(memOnly, 200L, 0L)) - taskMetrics0.updatedBlocks = Some(Seq(block0)) - taskMetrics1.updatedBlocks = Some(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 068e8397c89b..0b7483e3b06a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -32,12 +32,7 @@ import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage._ class JsonProtocolSuite extends SparkFunSuite { - - val jobSubmissionTime = 1421191042750L - val jobCompletionTime = 1421191296660L - - val executorAddedTime = 1421458410000L - val executorRemovedTime = 1421458922000L + import JsonProtocolSuite._ test("SparkListenerEvent") { val stageSubmitted = @@ -84,7 +79,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) @@ -142,7 +137,7 @@ 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, Seq.empty[AccumulableInfo]) testTaskEndReason(Success) testTaskEndReason(Resubmitted) testTaskEndReason(fetchFailed) @@ -167,8 +162,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)) @@ -227,7 +221,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") { @@ -273,14 +267,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) } @@ -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) @@ -478,14 +483,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!") @@ -544,7 +552,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) @@ -557,7 +564,7 @@ class JsonProtocolSuite extends SparkFunSuite { metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals) assertOptionEquals( metrics1.inputMetrics, metrics2.inputMetrics, assertInputMetricsEquals) - assertOptionEquals(metrics1.updatedBlocks, metrics2.updatedBlocks, assertBlocksEquals) + assertBlocksEquals(metrics1.updatedBlockStatuses, metrics2.updatedBlockStatuses) } private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { @@ -568,8 +575,8 @@ class JsonProtocolSuite extends SparkFunSuite { } private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) { - assert(metrics1.shuffleBytesWritten === metrics2.shuffleBytesWritten) - assert(metrics1.shuffleWriteTime === metrics2.shuffleWriteTime) + assert(metrics1.bytesWritten === metrics2.bytesWritten) + assert(metrics1.writeTime === metrics2.writeTime) } private def assertEquals(metrics1: InputMetrics, metrics2: InputMetrics) { @@ -601,7 +608,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), @@ -639,8 +646,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) { @@ -747,7 +753,8 @@ 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, 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 @@ -764,7 +771,6 @@ class JsonProtocolSuite extends SparkFunSuite { hasOutput: Boolean, hasRecords: Boolean = true) = { val t = new TaskMetrics - t.setHostname("localhost") t.setExecutorDeserializeTime(a) t.setExecutorRunTime(b) t.setResultSize(c) @@ -773,34 +779,31 @@ 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 = 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) + 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 - sw.incShuffleBytesWritten(a + b + c) - sw.incShuffleWriteTime(b + c + d) - sw.setShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) - t.shuffleWriteMetrics = Some(sw) + val sw = t.registerShuffleWriteMetrics() + 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 => + 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)) }.toSeq) t @@ -829,14 +832,16 @@ class 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 | } | ] | }, @@ -884,14 +889,16 @@ class 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 | } | ] | } @@ -922,21 +929,24 @@ class 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 | } | ] | } @@ -965,21 +975,24 @@ class 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 | } | ] | } @@ -1014,26 +1027,28 @@ class 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 | } | ] | }, | "Task Metrics": { - | "Host Name": "localhost", | "Executor Deserialize Time": 300, | "Executor Run Time": 400, | "Result Size": 500, @@ -1047,7 +1062,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, @@ -1101,26 +1116,28 @@ class 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 | } | ] | }, | "Task Metrics": { - | "Host Name": "localhost", | "Executor Deserialize Time": 300, | "Executor Run Time": 400, | "Result Size": 500, @@ -1185,26 +1202,28 @@ class 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 | } | ] | }, | "Task Metrics": { - | "Host Name": "localhost", | "Executor Deserialize Time": 300, | "Executor Run Time": 400, | "Result Size": 500, @@ -1276,17 +1295,19 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | }, @@ -1334,17 +1355,19 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | }, @@ -1408,17 +1431,19 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | }, @@ -1498,17 +1523,19 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | } @@ -1660,51 +1687,209 @@ 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": { - | "Host Name": "localhost", - | "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, - | "Deserialized": false, - | "Replication": 2 - | }, - | "Memory 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": 81, + | "Name": "$EXECUTOR_DESERIALIZE_TIME", + | "Update": 300, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 82, + | "Name": "$EXECUTOR_RUN_TIME", + | "Update": 400, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 83, + | "Name": "$RESULT_SIZE", + | "Update": 500, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 84, + | "Name": "$JVM_GC_TIME", + | "Update": 600, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 85, + | "Name": "$RESULT_SERIALIZATION_TIME", + | "Update": 700, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 86, + | "Name": "$MEMORY_BYTES_SPILLED", + | "Update": 800, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 87, + | "Name": "$DISK_BYTES_SPILLED", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 88, + | "Name": "$PEAK_EXECUTION_MEMORY", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 89, + | "Name": "$UPDATED_BLOCK_STATUSES", + | "Update": [ + | { + | "BlockID": "rdd_0_0", + | "Status": { + | "StorageLevel": { + | "UseDisk": true, + | "UseMemory": true, + | "Deserialized": false, + | "Replication": 2 + | }, + | "MemorySize": 0, + | "DiskSize": 0 + | } + | } + | ], + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 90, + | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 91, + | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 92, + | "Name": "${shuffleRead.REMOTE_BYTES_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 93, + | "Name": "${shuffleRead.LOCAL_BYTES_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 94, + | "Name": "${shuffleRead.FETCH_WAIT_TIME}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 95, + | "Name": "${shuffleRead.RECORDS_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 96, + | "Name": "${shuffleWrite.BYTES_WRITTEN}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 97, + | "Name": "${shuffleWrite.RECORDS_WRITTEN}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 98, + | "Name": "${shuffleWrite.WRITE_TIME}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 99, + | "Name": "${input.READ_METHOD}", + | "Update": "Hadoop", + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 100, + | "Name": "${input.BYTES_READ}", + | "Update": 2100, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 101, + | "Name": "${input.RECORDS_READ}", + | "Update": 21, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 102, + | "Name": "${output.WRITE_METHOD}", + | "Update": "Hadoop", + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 103, + | "Name": "${output.BYTES_WRITTEN}", + | "Update": 1200, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 104, + | "Name": "${output.RECORDS_WRITTEN}", + | "Update": 12, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 105, + | "Name": "$TEST_ACCUM", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | } + | ] + | } + | ] + |} + """.stripMargin + } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ccd3c34bb5c8..69d03bd50820 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -139,6 +139,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_"), 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 41799c596b6d..001e9c306ac4 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/datasources/SqlNewHadoopRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala index d45d2db62f3a..505a387491f2 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 { @@ -137,14 +136,17 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( // 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 +211,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 +234,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/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 0a818cc2c2a2..5c0465be0df6 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 6c7fa2eee5bf..db8edd169dcf 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/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..950dc7816241 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 /** @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils */ 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), internal = true) { def reset(): Unit = { this.value = param.zero @@ -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 cd5613692708..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) } } @@ -177,7 +176,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 +288,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 +330,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 +408,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/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 03d67c4e91f7..684dfe8dc8ab 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() } } 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/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index 9c258cb31f46..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 @@ -71,8 +71,7 @@ class UnsafeFixedWidthAggregationMapSuite taskAttemptId = Random.nextInt(10000), attemptNumber = 0, taskMemoryManager = taskMemoryManager, - metricsSystem = null, - internalAccumulators = Seq.empty)) + 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 95c9550aebb0..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 = Seq.empty)) + 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 9f09eb4429c1..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, @@ -127,7 +126,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/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..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 @@ -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,41 @@ 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() + // 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) + }, 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)) 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 eef3c1f3e34d..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 @@ -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.{SparkConf, SparkContext, SparkException, 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,11 @@ 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.map { case (id, update) => + new AccumulableInfo(id, "", Some(new LongSQLMetricValue(update)), + value = None, internal = true, countFailedValues = true) + }.toSeq) metrics } @@ -115,16 +120,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)) @@ -134,8 +139,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)) @@ -174,8 +179,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)) 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)