-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-20702][Core]TaskContextImpl.markTaskCompleted should not hide the original error #17942
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -115,26 +115,33 @@ private[spark] abstract class Task[T]( | |
| case t: Throwable => | ||
| e.addSuppressed(t) | ||
| } | ||
| context.markTaskCompleted(Some(e)) | ||
| throw e | ||
| } finally { | ||
| // Call the task completion callbacks. | ||
| context.markTaskCompleted() | ||
| try { | ||
| Utils.tryLogNonFatalError { | ||
| // Release memory used by this thread for unrolling blocks | ||
| SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP) | ||
| SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.OFF_HEAP) | ||
| // Notify any tasks waiting for execution memory to be freed to wake up and try to | ||
| // acquire memory again. This makes impossible the scenario where a task sleeps forever | ||
| // because there are no other tasks left to notify it. Since this is safe to do but may | ||
| // not be strictly necessary, we should revisit whether we can remove this in the future. | ||
| val memoryManager = SparkEnv.get.memoryManager | ||
| memoryManager.synchronized { memoryManager.notifyAll() } | ||
| } | ||
| // Call the task completion callbacks. If "markTaskCompleted" is called twice, the second | ||
| // one is no-op. | ||
| context.markTaskCompleted(None) | ||
|
Member
Author
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. Just add
Contributor
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. We end up calling
Member
Author
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.
Yes.
Then if
Contributor
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. What I meant was, when there is an exception is throw, there will be two invocations of This would be confusing to the listeners - no ?
Member
Author
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. @mridulm there is a |
||
| } finally { | ||
| // Though we unset the ThreadLocal here, the context member variable itself is still queried | ||
| // directly in the TaskRunner to check for FetchFailedExceptions. | ||
| TaskContext.unset() | ||
| try { | ||
| Utils.tryLogNonFatalError { | ||
| // Release memory used by this thread for unrolling blocks | ||
| SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP) | ||
| SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask( | ||
| MemoryMode.OFF_HEAP) | ||
| // Notify any tasks waiting for execution memory to be freed to wake up and try to | ||
| // acquire memory again. This makes impossible the scenario where a task sleeps forever | ||
| // because there are no other tasks left to notify it. Since this is safe to do but may | ||
| // not be strictly necessary, we should revisit whether we can remove this in the | ||
| // future. | ||
| val memoryManager = SparkEnv.get.memoryManager | ||
| memoryManager.synchronized { memoryManager.notifyAll() } | ||
| } | ||
| } finally { | ||
| // Though we unset the ThreadLocal here, the context member variable itself is still | ||
| // queried directly in the TaskRunner to check for FetchFailedExceptions. | ||
| TaskContext.unset() | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -55,14 +55,16 @@ class TaskCompletionListenerException( | |
| extends RuntimeException { | ||
|
|
||
| override def getMessage: String = { | ||
| if (errorMessages.size == 1) { | ||
|
Member
Author
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. The problem here is is equivalent to which is not the intention.
Contributor
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. Good catch !
Member
Author
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. I guess it's because it's not a class name. cc @rxin since you added this file.
Contributor
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. Aside: Was just curious about the naming - interesting. Is this common pattern in spark code ?
Contributor
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. It's a common pattern in scala.
Contributor
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. Thx for clarifying ! |
||
| errorMessages.head | ||
| } else { | ||
| errorMessages.zipWithIndex.map { case (msg, i) => s"Exception $i: $msg" }.mkString("\n") | ||
| } + | ||
| previousError.map { e => | ||
| val listenerErrorMessage = | ||
| if (errorMessages.size == 1) { | ||
| errorMessages.head | ||
| } else { | ||
| errorMessages.zipWithIndex.map { case (msg, i) => s"Exception $i: $msg" }.mkString("\n") | ||
| } | ||
| val previousErrorMessage = previousError.map { e => | ||
| "\n\nPrevious exception in task: " + e.getMessage + "\n" + | ||
| e.getStackTrace.mkString("\t", "\n\t", "") | ||
| }.getOrElse("") | ||
| listenerErrorMessage + previousErrorMessage | ||
| } | ||
| } | ||
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.
Missed this comment.
LGTM. Thanks for clarifying @zsxwing