-
Notifications
You must be signed in to change notification settings - Fork 28.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-732][SPARK-3628][CORE][RESUBMIT] eliminate duplicate update on accmulator #2524
Changes from 12 commits
67593d2
138f9b3
5cf586f
f74266b
88d1f03
21b6840
1e9e14d
84570d2
83b75f8
2b2e8cf
6b0aff9
02261b8
b233737
1433e6f
701a1e8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
package org.apache.spark | ||
|
||
import java.io.{ObjectInputStream, Serializable} | ||
import java.util.concurrent.atomic.AtomicLong | ||
|
||
import scala.collection.generic.Growable | ||
import scala.collection.mutable.Map | ||
|
@@ -228,6 +229,7 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa | |
*/ | ||
class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T], name: Option[String]) | ||
extends Accumulable[T,T](initialValue, param, name) { | ||
|
||
def this(initialValue: T, param: AccumulatorParam[T]) = this(initialValue, param, None) | ||
} | ||
|
||
|
@@ -252,10 +254,9 @@ private object Accumulators { | |
val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]() | ||
var lastId: Long = 0 | ||
|
||
def newId: Long = synchronized { | ||
lastId += 1 | ||
lastId | ||
} | ||
private val nextAccumID = new AtomicLong(0) | ||
|
||
def newId: Long = nextAccumID.getAndIncrement | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit: this should be called |
||
|
||
def register(a: Accumulable[_, _], original: Boolean): Unit = synchronized { | ||
if (original) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,7 +21,7 @@ import java.io.NotSerializableException | |
import java.util.Properties | ||
import java.util.concurrent.atomic.AtomicInteger | ||
|
||
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} | ||
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack, ListBuffer} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why was ListBuffer added? |
||
import scala.concurrent.Await | ||
import scala.concurrent.duration._ | ||
import scala.language.postfixOps | ||
|
@@ -449,7 +449,6 @@ class DAGScheduler( | |
} | ||
// data structures based on StageId | ||
stageIdToStage -= stageId | ||
|
||
logDebug("After removal of stage %d, remaining stages = %d" | ||
.format(stageId, stageIdToStage.size)) | ||
} | ||
|
@@ -901,6 +900,33 @@ class DAGScheduler( | |
} | ||
} | ||
|
||
private def updateAccumulator(event: CompletionEvent): Unit = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Call this updateAccumulators and add a comment saying |
||
val task = event.task | ||
val stage = stageIdToStage(task.stageId) | ||
if (event.accumUpdates != null) { | ||
try { | ||
Accumulators.add(event.accumUpdates) | ||
event.accumUpdates.foreach { case (id, partialValue) => | ||
val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] | ||
// To avoid UI cruft, ignore cases where value wasn't updated | ||
if (acc.name.isDefined && partialValue != acc.zero) { | ||
val name = acc.name.get | ||
val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) | ||
val stringValue = Accumulators.stringifyValue(acc.value) | ||
stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) | ||
event.taskInfo.accumulables += | ||
AccumulableInfo(id, name, Some(stringPartialValue), stringValue) | ||
} | ||
} | ||
} 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) | ||
} | ||
} | ||
} | ||
|
||
/** | ||
* Responds to a task finishing. This is called inside the event loop so it assumes that it can | ||
* modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. | ||
|
@@ -941,27 +967,6 @@ class DAGScheduler( | |
} | ||
event.reason match { | ||
case Success => | ||
if (event.accumUpdates != null) { | ||
try { | ||
Accumulators.add(event.accumUpdates) | ||
event.accumUpdates.foreach { case (id, partialValue) => | ||
val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] | ||
// To avoid UI cruft, ignore cases where value wasn't updated | ||
if (acc.name.isDefined && partialValue != acc.zero) { | ||
val name = acc.name.get | ||
val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) | ||
val stringValue = Accumulators.stringifyValue(acc.value) | ||
stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) | ||
event.taskInfo.accumulables += | ||
AccumulableInfo(id, name, Some(stringPartialValue), stringValue) | ||
} | ||
} | ||
} 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) | ||
} | ||
} | ||
listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, | ||
event.reason, event.taskInfo, event.taskMetrics)) | ||
stage.pendingTasks -= task | ||
|
@@ -970,6 +975,7 @@ class DAGScheduler( | |
stage.resultOfJob match { | ||
case Some(job) => | ||
if (!job.finished(rt.outputId)) { | ||
updateAccumulator(event) | ||
job.finished(rt.outputId) = true | ||
job.numFinished += 1 | ||
// If the whole job has finished, remove it | ||
|
@@ -994,6 +1000,7 @@ class DAGScheduler( | |
} | ||
|
||
case smt: ShuffleMapTask => | ||
updateAccumulator(event) | ||
val status = event.result.asInstanceOf[MapStatus] | ||
val execId = status.location.executorId | ||
logDebug("ShuffleMapTask finished on " + execId) | ||
|
@@ -1082,7 +1089,6 @@ class DAGScheduler( | |
} | ||
failedStages += failedStage | ||
failedStages += mapStage | ||
|
||
// Mark the map whose fetch failed as broken in the map stage | ||
if (mapId != -1) { | ||
mapStage.removeOutputLoc(mapId, bmAddress) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1228,6 +1228,11 @@ interface to accumulate data where the resulting type is not the same as the ele | |
a list by collecting together elements), and the `SparkContext.accumulableCollection` method for accumulating | ||
common Scala collection types. | ||
|
||
<b>Only when the accumulator operation is executed within an | ||
action</b>, Spark guarantees that the operation will only be applied when the task is successfully finished for | ||
the first time, i.e. the restarted task will not update the value. In transformations, users should be aware of that | ||
the accumulator value would be updated as long as the task is executed. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for adding this, but it's better to tweak the wording a bit like this: "For accumulator updates performed inside actions only, Spark guarantees that each task's update to the accumulator will only be applied once, i.e. restarted tasks will not update the value. In transformations, users should be aware of that each task's update may be applied more than once if tasks or job stages are re-executed." Also, move this paragraph below the language-specific |
||
|
||
</div> | ||
|
||
<div data-lang="java" markdown="1"> | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You can delete the lastId variable if you use AtomicLong.