-
Notifications
You must be signed in to change notification settings - Fork 28k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-12895][SPARK-12896] Migrate TaskMetrics to accumulators
The high level idea is that instead of having the executors send both accumulator updates and TaskMetrics, we should have them send only accumulator updates. This eliminates the need to maintain both code paths since one can be implemented in terms of the other. This effort is split into two parts: **SPARK-12895: Implement TaskMetrics using accumulators.** TaskMetrics is basically just a bunch of accumulable fields. This patch makes TaskMetrics a syntactic wrapper around a collection of accumulators so we don't need to send TaskMetrics from the executors to the driver. **SPARK-12896: Send only accumulator updates to the driver.** Now that TaskMetrics are expressed in terms of accumulators, we can capture all TaskMetrics values if we just send accumulator updates from the executors to the driver. This completes the parent issue SPARK-10620. While an effort has been made to preserve as much of the public API as possible, there were a few known breaking DeveloperApi changes that would be very awkward to maintain. I will gather the full list shortly and post it here. Note: This was once part of #10717. This patch is split out into its own patch from there to make it easier for others to review. Other smaller pieces of already been merged into master. Author: Andrew Or <andrew@databricks.com> Closes #10835 from andrewor14/task-metrics-use-accums.
- Loading branch information
Showing
70 changed files
with
3,012 additions
and
1,141 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -35,40 +35,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, | ||
This comment has been minimized.
Sorry, something went wrong.
This comment has been minimized.
Sorry, something went wrong.
JoshRosen
Contributor
|
||
param: AccumulableParam[R, T], | ||
val name: Option[String], | ||
internal: Boolean) | ||
internal: Boolean, | ||
private[spark] 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 /* internal */) | ||
|
||
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 | ||
|
@@ -77,6 +104,17 @@ 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 | ||
|
@@ -106,7 +144,7 @@ class Accumulable[R, T] private[spark] ( | |
def merge(term: R) { 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) { | ||
|
@@ -128,7 +166,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) { | ||
|
@@ -139,22 +177,24 @@ 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 | ||
} | ||
def setValue(newValue: R): Unit = { value_ = newValue } | ||
|
||
/** | ||
* Set the accumulator's value. For internal use only. | ||
*/ | ||
private[spark] def setValueAny(newValue: Any): Unit = { setValue(newValue.asInstanceOf[R]) } | ||
|
||
// Called by Java when deserializing an object | ||
private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { | ||
in.defaultReadObject() | ||
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) | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Seems this needs to be
@transient private val
.