yaooqinn commented on code in PR #44575:
URL: https://github.com/apache/spark/pull/44575#discussion_r1441226704
##########
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala:
##########
@@ -201,6 +201,16 @@ private[spark] object HiveUtils extends Logging {
.booleanConf
.createWithDefault(true)
+ val HIVE_THRIFT_SERVER_EXIT_ON_ERROR =
+ buildConf("spark.sql.hive.thriftServer.exitOnThriftCLIServiceError")
+ .internal()
+ .doc("When true, System.exit(-1) will be called if the underlying
TServer/HTTPServer " +
+ "encounters an error when start serving. When false, the error will be
thrown which" +
+ "enables error handling of the DeveloperApi
`HiveThriftServer2.startWithContext`")
+ .version("4.0.0")
+ .booleanConf
+ .createWithDefault(true)
Review Comment:
[For
instance](https://github.com/apache/spark/pull/44575/files#diff-b549322f1450668610a56a4ebed7f2bb284b8aaa66d90561bb75634a9a9f4d1cR140),
when we retry the thrift server on the same sc. If this is true, sc will be
stopped. And weird things follow in the next retries
```
[info]
org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextInHttpSuite
*** ABORTED *** (151 milliseconds)
[info] java.lang.IllegalStateException: Cannot call methods on a stopped
SparkContext.
[info] This stopped SparkContext was created at:
[info]
[info]
org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextInHttpSuite.beforeAll(ThriftServerWithSparkContextSuite.scala:279)
[info]
org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212)
[info] org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
[info] org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
[info] org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:69)
[info]
org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:321)
[info]
org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:517)
[info] sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:414)
[info] java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
[info]
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
[info]
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
[info] java.base/java.lang.Thread.run(Thread.java:840)
[info]
[info] The currently active SparkContext was created at:
[info]
[info] (No active SparkContext.)
[info] at
org.apache.spark.SparkContext.assertNotStopped(SparkContext.scala:122)
[info] at org.apache.spark.sql.SparkSession.<init>(SparkSession.scala:115)
[info] at
org.apache.spark.sql.SparkSession.newSession(SparkSession.scala:274)
[info] at
org.apache.spark.sql.hive.thriftserver.SharedThriftServer.startThriftServer(SharedThriftServer.scala:130)
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]