Skip to content

Conversation

@tigerquoll
Copy link
Contributor

[SPARK-4787] Stop SparkContext properly if an exception occurs during DAGscheduler initialization.

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Style nit: you can use string interpolation instead of String.format, which will allow the new SparkException to fit on the same line as throw:

throw new SparkException(s"DAGScheduler cannot be initialized due to ${e.getMessage}")

However, I'd prefer to call the two-argument constructor which takes the cause as second argument, since this will lead to more informative stacktraces:

throw new SparkException("Error while constructing DAGScheduler", e)

@JoshRosen
Copy link
Contributor

Jenkins, this is ok to test.

@SparkQA
Copy link

SparkQA commented Dec 26, 2014

Test build #24838 has started for PR 3809 at commit 2172578.

  • This patch merges cleanly.

@JoshRosen
Copy link
Contributor

This is a nice fix.

Resource leaks when SparkContext's constructor throws exceptions have been a longstanding issue. I first ran across the issue while adding logic to detect whether a SparkContext was already running when attempting to create a new one (SPARK-4180). In that case, I ran into some issues because I wanted to effectively make the entire constructor synchronized on a static object, but this was hard because there wasn't an explicit constructor method. We could have tried to wrap the entire "implicit constructor" in a try-finally block, but this would require us to re-organize a huge amount of code and change many vals into vars. I had an alternative proposal to move the dependency-creation into the SparkContext companion object and pass in a SparkContextDependencies object into SparkContext's constructors, which would solve this issue more generally (but it's a much larger change). See the PR description at #3121 for more details.

Barring a big restructuring of SparkContext's constructor, though, small fixes like this are welcome. Therefore, this looks good to me.

@JoshRosen
Copy link
Contributor

By the way, I left a comment over on JIRA about the scope of the SPARK-4787 JIRA. If we merge this PR as-is, without adding more try-catches for other statements that could throw exceptions, then I think we should revise that JIRA to describe only the fix implemented here (error-catching for DAGScheduler errors) and should convert it into a subtask of SPARK-4180.

@SparkQA
Copy link

SparkQA commented Dec 26, 2014

Test build #24838 has finished for PR 3809 at commit 2172578.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24838/
Test PASSed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, do you think this should be in a try-finally block so that we don't swallow the useful "DAGScheduler could not be initialized" exception if the stop() call somehow fails?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Excellent idea Josh.

@SparkQA
Copy link

SparkQA commented Dec 30, 2014

Test build #24873 has started for PR 3809 at commit 5661e01.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Dec 30, 2014

Test build #24873 has finished for PR 3809 at commit 5661e01.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24873/
Test PASSed.

@JoshRosen
Copy link
Contributor

Alright, I've re-scoped SPARK-4787 to only cover the changes here and made it into a subtask of the broader SPARK-4194 issue. So as "not to let the perfect be the enemy of the good", I'm going to merge this now and we can address the broader cleanup issues in later PRs. Thanks for the fix!

asfgit pushed a commit that referenced this pull request Jan 4, 2015
Author: Dale <[email protected]>

Closes #3809 from tigerquoll/SPARK-4787 and squashes the following commits:

5661e01 [Dale] [SPARK-4787] Ensure that call to stop() doesn't lose the exception by using a finally block.
2172578 [Dale] [SPARK-4787] Stop context properly if an exception occurs during DAGScheduler initialization.

(cherry picked from commit 3fddc94)
Signed-off-by: Josh Rosen <[email protected]>
asfgit pushed a commit that referenced this pull request Jan 4, 2015
Author: Dale <[email protected]>

Closes #3809 from tigerquoll/SPARK-4787 and squashes the following commits:

5661e01 [Dale] [SPARK-4787] Ensure that call to stop() doesn't lose the exception by using a finally block.
2172578 [Dale] [SPARK-4787] Stop context properly if an exception occurs during DAGScheduler initialization.

(cherry picked from commit 3fddc94)
Signed-off-by: Josh Rosen <[email protected]>
@asfgit asfgit closed this in 3fddc94 Jan 4, 2015
asfgit pushed a commit that referenced this pull request Jan 4, 2015
Author: Dale <[email protected]>

Closes #3809 from tigerquoll/SPARK-4787 and squashes the following commits:

5661e01 [Dale] [SPARK-4787] Ensure that call to stop() doesn't lose the exception by using a finally block.
2172578 [Dale] [SPARK-4787] Stop context properly if an exception occurs during DAGScheduler initialization.

(cherry picked from commit 3fddc94)
Signed-off-by: Josh Rosen <[email protected]>
@JoshRosen
Copy link
Contributor

Merged into master (1.3.0), branch-1.2 (1.2.1), branch-1.1 (1.1.2), and branch-1.0 (1.0.3).

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.

4 participants