-
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-611] Display executor thread dumps in web UI #2944
Conversation
This patch allows executor thread dumps to be viewed in the Spark web UI. Thread dumps obtained from Thread.getAllStackTraces() are piggybacked on the periodic executor -> driver heartbeats. JobProgressListener stores these heartbeats for display in the UI. One current limitation is that the driver thread dumps are not viewable except when running in local mode.
@@ -47,7 +47,7 @@ private[spark] class LocalActor( | |||
|
|||
private var freeCores = totalCores | |||
|
|||
private val localExecutorId = "localhost" | |||
private val localExecutorId = "<driver>" |
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.
While working on this, I noticed that the links to view the driver thread dumps weren't working. In local
mode, it looks like we weren't using the same name for the local executor id in this Executor and in the driver's SparkEnv, so I fixed that here.
Test build #22228 has started for PR 2944 at commit
|
One subtle issue that I've run into is that the driver always runs a block manager but only runs an Executor in local mode. So, the "executors" tab in the web UI is slightly misleading when running in a cluster mode, since the driver doesn't run a regular executor. It would be nice to have driver / application thread dumps in this UI, too, so I wonder if there's a clean way to fix this. |
Executor IDs are strings, so I should probably check whether they'll need to be url-encoded; I guess this depends on which components create these strings. |
Test build #22228 has finished for PR 2944 at commit
|
Test FAILed. |
Test build #22301 has started for PR 2944 at commit
|
Test FAILed. |
jenkins, test this please |
Test build #22305 has started for PR 2944 at commit
|
Test build #22305 has finished for PR 2944 at commit
|
Test PASSed. |
Wow, awesome!! |
This is even easier to read than the raw jstack output |
@JoshRosen This is super awesome ! |
It looks like executorIds are assigned by the cluster manager, so in principle they could be arbitrary strings but in practice they seem to not contain special characters that would need special escaping (such as spaces). The application master log viewer (LogPage.scala) doesn't perform any URL-encoding or escaping of executorIds, so I'm not sure that we need to do it here. |
Do you know how large the threadDump is typically ? I'm concerned this might make the heartbeat too large |
The other idea I had was that we could just open a port on the executor and have a web ui on it. This could also display the executor's stderr (Which is very painful to get to right now) and have links to get thread stack trace, memory histo etc. However this might be a larger change |
@shivaram That's a good point RE: the size of the thread dumps. I can now imagine problems where a thread-leak in an executor causes the heartbeat to become huge and leads to a job failure when the heartbeat exceeds the Akka frame size. |
Test build #22383 has started for PR 2944 at commit
|
I like the idea of running a separate UI server on the executor, but this seems like a much more involved change that will take a lot more design review. For example, we'd have to consider how those web UIs will be secured, which ports they will bind to, etc. As a shorter-term fix, how about de-coupling the thread dumps from the heartbeats so that huge thread dumps won't cause heartbeats to be lost? If we do this, I might be able to add a driver -> executor RPC path to allow thread-dumps to be triggered from the web UI. |
Yes - I think having a separate RPC sounds good for now. |
Upon closer inspection, there's not a general driver -> executor RPC path that I can use to send arbitrary Akka messages to executors. To keep this PR simple and narrow in scope, I'm just going to add a separate RPC. |
Test build #22383 has finished for PR 2944 at commit
|
Test PASSed. |
Test build #22616 has finished for PR 2944 at commit
|
Test FAILed. |
retest this please |
Test build #22644 has started for PR 2944 at commit
|
Test build #22644 has finished for PR 2944 at commit
|
Test PASSed. |
Conflicts: core/src/main/scala/org/apache/spark/SparkContext.scala
I've pushed a new commit to fix the merge conflict here. Could someone review this latest revision? |
Test build #22744 has started for PR 2944 at commit
|
Test build #22744 has finished for PR 2944 at commit
|
Test PASSed. |
@@ -131,7 +131,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | |||
// Create a new ActorSystem using driver's Spark properties to run the backend. | |||
val driverConf = new SparkConf().setAll(props) | |||
val (actorSystem, boundPort) = AkkaUtils.createActorSystem( | |||
"sparkExecutor", hostname, port, driverConf, new SecurityManager(driverConf)) | |||
SparkEnv.executorActorSystemName, |
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.
Nice, we had this variable but we just never used it.
Hey I just left a few relatively minor comments but the overall approach looks good. It's great to see that we're not doing this through the block manager interface. In the long run we may want to move more of the existing stuff into |
Thread.getAllStackTraces.toArray.sortBy(_._1.getId).map { | ||
case (thread, stackElements) => | ||
val stackTrace = stackElements.map(_.toString).mkString("\n") | ||
ThreadStackTrace(thread.getId, thread.getName, thread.getState.toString, stackTrace) |
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.
Just spotted a consistency issue here: thread.getState
is grabbed at a different time than the stack elements.
- Rename ThreadDumpPage -> ExecutorThreadDumpPage - Make page private[ui] - Make TriggerThreadDump into a case object - Rename fields in ThreadStackTrace - Use ThreadMxBean to obtain thread dumps instead of Thread.getAllStackTraces() - Remove documentation of spark.ui.threadDumpsEnabled configuration, but leave the option as an internal configuration - Guard against exceptions in SparkContext.getExecutorThreadDump() - Disable thread dump page and button in history server.
Alright, I took another pass on this:
|
Test build #22831 has started for PR 2944 at commit
|
Test build #22831 has finished for PR 2944 at commit
|
Test PASSed. |
LGTM, I can't wait to use this feature myself! |
Ok I'm merging this. |
This patch allows executor thread dumps to be collected on-demand and viewed in the Spark web UI. The thread dumps are collected using Thread.getAllStackTraces(). To allow remote thread dumps to be triggered from the web UI, I added a new `ExecutorActor` that runs inside of the Executor actor system and responds to RPCs from the driver. The driver's mechanism for obtaining a reference to this actor is a little bit hacky: it uses the block manager master actor to determine the host/port of the executor actor systems in order to construct ActorRefs to ExecutorActor. Unfortunately, I couldn't find a much cleaner way to do this without a big refactoring of the executor -> driver communication. Screenshots: ![image](https://cloud.githubusercontent.com/assets/50748/4781793/7e7a0776-5cbf-11e4-874d-a91cd04620bd.png) ![image](https://cloud.githubusercontent.com/assets/50748/4781794/8bce76aa-5cbf-11e4-8d13-8477748c9f7e.png) ![image](https://cloud.githubusercontent.com/assets/50748/4781797/bd11a8b8-5cbf-11e4-9ad7-a7459467ec8e.png) Author: Josh Rosen <[email protected]> Closes #2944 from JoshRosen/jstack-in-web-ui and squashes the following commits: 3c21a5d [Josh Rosen] Address review comments: 880f7f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui f719266 [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui 19707b0 [Josh Rosen] Add one comment. 127a130 [Josh Rosen] Update to use SparkContext.DRIVER_IDENTIFIER b8e69aa [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui 3dfc2d4 [Josh Rosen] Add missing file. bc1e675 [Josh Rosen] Undo some leftover changes from the earlier approach. f4ac1c1 [Josh Rosen] Switch to on-demand collection of thread dumps dfec08b [Josh Rosen] Add option to disable thread dumps in UI. 4c87d7f [Josh Rosen] Use separate RPC for sending thread dumps. 2b8bdf3 [Josh Rosen] Enable thread dumps from the driver when running in non-local mode. cc3e6b3 [Josh Rosen] Fix test code in DAGSchedulerSuite. 87b8b65 [Josh Rosen] Add new listener event for thread dumps. 8c10216 [Josh Rosen] Add missing file. 0f198ac [Josh Rosen] [SPARK-611] Display executor thread dumps in web UI
This patch allows executor thread dumps to be collected on-demand and viewed in the Spark web UI.
The thread dumps are collected using Thread.getAllStackTraces(). To allow remote thread dumps to be triggered from the web UI, I added a new
ExecutorActor
that runs inside of the Executor actor system and responds to RPCs from the driver. The driver's mechanism for obtaining a reference to this actor is a little bit hacky: it uses the block manager master actor to determine the host/port of the executor actor systems in order to construct ActorRefs to ExecutorActor. Unfortunately, I couldn't find a much cleaner way to do this without a big refactoring of the executor -> driver communication.Screenshots: