-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-16451][repl] Fail shell if SparkSession fails to start. #21368
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
Closed
Closed
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -44,6 +44,7 @@ object Main extends Logging { | |
| var interp: SparkILoop = _ | ||
|
|
||
| private var hasErrors = false | ||
| private var isShellSession = false | ||
|
|
||
| private def scalaOptionError(msg: String): Unit = { | ||
| hasErrors = true | ||
|
|
@@ -53,6 +54,7 @@ object Main extends Logging { | |
| } | ||
|
|
||
| def main(args: Array[String]) { | ||
| isShellSession = true | ||
| doMain(args, new SparkILoop) | ||
| } | ||
|
|
||
|
|
@@ -79,44 +81,50 @@ object Main extends Logging { | |
| } | ||
|
|
||
| def createSparkSession(): SparkSession = { | ||
| val execUri = System.getenv("SPARK_EXECUTOR_URI") | ||
| conf.setIfMissing("spark.app.name", "Spark shell") | ||
| // SparkContext will detect this configuration and register it with the RpcEnv's | ||
| // file server, setting spark.repl.class.uri to the actual URI for executors to | ||
| // use. This is sort of ugly but since executors are started as part of SparkContext | ||
| // initialization in certain cases, there's an initialization order issue that prevents | ||
| // this from being set after SparkContext is instantiated. | ||
| conf.set("spark.repl.class.outputDir", outputDir.getAbsolutePath()) | ||
| if (execUri != null) { | ||
| conf.set("spark.executor.uri", execUri) | ||
| } | ||
| if (System.getenv("SPARK_HOME") != null) { | ||
| conf.setSparkHome(System.getenv("SPARK_HOME")) | ||
| } | ||
| try { | ||
| val execUri = System.getenv("SPARK_EXECUTOR_URI") | ||
| conf.setIfMissing("spark.app.name", "Spark shell") | ||
| // SparkContext will detect this configuration and register it with the RpcEnv's | ||
| // file server, setting spark.repl.class.uri to the actual URI for executors to | ||
| // use. This is sort of ugly but since executors are started as part of SparkContext | ||
| // initialization in certain cases, there's an initialization order issue that prevents | ||
| // this from being set after SparkContext is instantiated. | ||
| conf.set("spark.repl.class.outputDir", outputDir.getAbsolutePath()) | ||
| if (execUri != null) { | ||
| conf.set("spark.executor.uri", execUri) | ||
| } | ||
| if (System.getenv("SPARK_HOME") != null) { | ||
| conf.setSparkHome(System.getenv("SPARK_HOME")) | ||
| } | ||
|
|
||
| val builder = SparkSession.builder.config(conf) | ||
| if (conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase(Locale.ROOT) == "hive") { | ||
| if (SparkSession.hiveClassesArePresent) { | ||
| // In the case that the property is not set at all, builder's config | ||
| // does not have this value set to 'hive' yet. The original default | ||
| // behavior is that when there are hive classes, we use hive catalog. | ||
| sparkSession = builder.enableHiveSupport().getOrCreate() | ||
| logInfo("Created Spark session with Hive support") | ||
| val builder = SparkSession.builder.config(conf) | ||
| if (conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase(Locale.ROOT) == "hive") { | ||
| if (SparkSession.hiveClassesArePresent) { | ||
| // In the case that the property is not set at all, builder's config | ||
| // does not have this value set to 'hive' yet. The original default | ||
| // behavior is that when there are hive classes, we use hive catalog. | ||
| sparkSession = builder.enableHiveSupport().getOrCreate() | ||
| logInfo("Created Spark session with Hive support") | ||
| } else { | ||
| // Need to change it back to 'in-memory' if no hive classes are found | ||
| // in the case that the property is set to hive in spark-defaults.conf | ||
| builder.config(CATALOG_IMPLEMENTATION.key, "in-memory") | ||
| sparkSession = builder.getOrCreate() | ||
| logInfo("Created Spark session") | ||
| } | ||
| } else { | ||
| // Need to change it back to 'in-memory' if no hive classes are found | ||
| // in the case that the property is set to hive in spark-defaults.conf | ||
| builder.config(CATALOG_IMPLEMENTATION.key, "in-memory") | ||
| // In the case that the property is set but not to 'hive', the internal | ||
| // default is 'in-memory'. So the sparkSession will use in-memory catalog. | ||
| sparkSession = builder.getOrCreate() | ||
| logInfo("Created Spark session") | ||
| } | ||
| } else { | ||
| // In the case that the property is set but not to 'hive', the internal | ||
| // default is 'in-memory'. So the sparkSession will use in-memory catalog. | ||
| sparkSession = builder.getOrCreate() | ||
| logInfo("Created Spark session") | ||
| sparkContext = sparkSession.sparkContext | ||
| sparkSession | ||
| } catch { | ||
| case e: Exception if isShellSession => | ||
| logError("Failed to initialize Spark session.", e) | ||
|
Member
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. @vanzin, seem
Contributor
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 exception is being printed as part of the |
||
| sys.exit(1) | ||
| } | ||
| sparkContext = sparkSession.sparkContext | ||
| sparkSession | ||
| } | ||
|
|
||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
the call flow seems to be changed here? I think this line is meant to be inside the handling of Py4JError?
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.
This is intentional to avoid the python exception being unreadable (see commit description).
The actual flow logic is the same.