Skip to content

Conversation

@xuzhongxing
Copy link

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@xuzhongxing xuzhongxing changed the title Fix spark driver hang in mesos fine-grained mode [SPARK-3005] Fix spark driver hang in mesos fine-grained mode Aug 14, 2014
@pwendell
Copy link
Contributor

I commented on the JIRA - but we already have code that handles the fact that cancellation is not supported in Mesos. It's likely this is related to some other type of error.

@SparkQA
Copy link

SparkQA commented Sep 5, 2014

Can one of the admins verify this patch?

@andrewor14
Copy link
Contributor

@xuzhongxing I followed the conversation on the JIRA and it looks like we still don't have a good idea of why Spark driver is hanging. Although we have a fix that makes the problem go away, the root cause is probably deeper, and the behavior you observed on fine-grained mode is just a symptom.

My guess is that when an UnsupportedOperationException is thrown when we try to killTask, we never end up posting the job end event to the listeners. This may be a behavior introduced in #1219. @kayousterhout Can you comment on this? Any thoughts on why DAGScheduler hangs if we don't post a job end event?

@kayousterhout
Copy link
Contributor

@andrewor14 I think you're right that there's a deeper problem here. I haven't tested this but here's what I think is going on:

(1) In TaskSchedulerImpl.cancelTasks(), the killTask call throws an unsupported operation exception, as is logged (https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L194). As a result, tsm.abort() never gets called. So, the TaskSetManager still thinks everything is hunky dory.
(2) Slowly the rest of the tasks fail, triggering the handleFailedTask() code in TaskSetManager. The TSM doesn't realize the task set is effectively dead because abort() was never called.
(3) Now, what I would expect to happen is that the code here:https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L605 would trigger the task to be re-launched. Eventually, a task would fail 4 times and the stage would get killed. This isn't exactly the right behavior, but still wouldn't lead to a hang. It might be good to understand why that isn't happening.

Regardless of what's going on with (3), I think the right way to fix this is to move the tsm.abort() call here: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L196 up to before we try to kill the task. That way, regardless of whether killTask() is successful, we'll mark the task set as aborted and send all the appropriate events.

Also, whoever fixes this should definitely add a unit test!! It would be great to add a short unit test to show the problem first, so it's easier for others to reproduce, and then deal with the fix.

@pwendell
Copy link
Contributor

@kayousterhout thanks for the thorough analysis. Do you have any thoughts on just defining killTasks to be "best effort"? I think that would generally simplify the code a lot here.

@kayousterhout
Copy link
Contributor

This seems like it could be ok -- my only concern is about the semantics of
when we tell the user we've killed their job. Currently I think we invoke
a callback on JobWaiter saying that the job has been killed, iff the
schedulerBackend implements killTask. So, if we make killTask() best
effort, the semantics of that callback will change.

On Mon, Sep 15, 2014 at 10:06 PM, Patrick Wendell [email protected]
wrote:

@kayousterhout https://github.com/kayousterhout thanks for the thorough
analysis. Do you have any thoughts on just defining killTasks to be "best
effort"? I think that would generally simplify the code a lot here.


Reply to this email directly or view it on GitHub
#1940 (comment).

@pwendell
Copy link
Contributor

Yeah I think we should just change it to say that the kill request has been acknowledge, but since killing as asynchronous and best-effort, it may not have stopped executing. The semantics are already somewhat weird, because right now users will get that message even if tasks from their job are still running (since it's asynchronous).

@tnachen
Copy link
Contributor

tnachen commented Sep 19, 2014

Just chiming in about the two different fixes about the killTask, where this PR does nothing while the Brenden's PR #2453 calls the Mesos driver kill task.
I think Brenden's approach is correct, since we don't know if the killTask was called because the task failed to launch or was actually being cancelled and Mesos task is already running.
On Mesos side if you call killTask on a non-existing Task all you get is a LOG(WARNING).

@andrewor14
Copy link
Contributor

Hey @xuzhongxing I think this is resolved in #2453. Would you mind closing this issue?

@tnachen
Copy link
Contributor

tnachen commented Nov 6, 2014

Please close this PR as this is no longer needed

@xuzhongxing xuzhongxing closed this Nov 7, 2014
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants