Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,7 @@ object CheckConnectJvmClientCompatibility {
ProblemFilters.exclude[Problem]("org.apache.spark.sql.SparkSession.executeCommand"),
ProblemFilters.exclude[Problem]("org.apache.spark.sql.SparkSession.readStream"),
ProblemFilters.exclude[Problem]("org.apache.spark.sql.SparkSession.this"),
ProblemFilters.exclude[Problem]("org.apache.spark.sql.SparkSession.stop"),

// RuntimeConfig
ProblemFilters.exclude[Problem]("org.apache.spark.sql.RuntimeConfig.this"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,18 +28,20 @@ object SparkConnectServer extends Logging {
// Set the active Spark Session, and starts SparkEnv instance (via Spark Context)
logInfo("Starting Spark session.")
val session = SparkSession.builder.getOrCreate()
var exitCode = 0
try {
try {
SparkConnectService.start()
logInfo("Spark Connect server started.")
} catch {
case e: Exception =>
logError("Error starting Spark Connect server", e)
System.exit(-1)
exitCode = -1
System.exit(exitCode)
}
SparkConnectService.server.awaitTermination()
} finally {
session.stop()
session.stop(exitCode)
}
}
}
11 changes: 10 additions & 1 deletion sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Original file line number Diff line number Diff line change
Expand Up @@ -753,14 +753,23 @@ class SparkSession private(
// scalastyle:on

/**
* Stop the underlying `SparkContext`.
* Stop the underlying `SparkContext` with default exit code 0.
*
* @since 2.0.0
Copy link
Member

Choose a reason for hiding this comment

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

You should have the new docs for the one for exitCode with new @since

Copy link
Contributor Author

Choose a reason for hiding this comment

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

How about current?

*/
def stop(): Unit = {
sparkContext.stop()
}

/**
* Stop the underlying `SparkContext` with exit code that will passed to scheduler backend.
*
* @since 3.5.0
*/
def stop(exitCode: Int): Unit = {
sparkContext.stop(exitCode)
}

/**
* Synonym for `stop()`.
*
Expand Down