Skip to content

Commit bbb87b3

Browse files
zuotingbinggatorsmile
authored andcommitted
[SPARK-22837][SQL] Session timeout checker does not work in SessionManager.
## What changes were proposed in this pull request? Currently we do not call the `super.init(hiveConf)` in `SparkSQLSessionManager.init`. So we do not load the config `HIVE_SERVER2_SESSION_CHECK_INTERVAL HIVE_SERVER2_IDLE_SESSION_TIMEOUT HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION` , which cause the session timeout checker does not work. ## How was this patch tested? manual tests Author: zuotingbing <[email protected]> Closes #20025 from zuotingbing/SPARK-22837.
1 parent 8c273b4 commit bbb87b3

File tree

1 file changed

+1
-15
lines changed

1 file changed

+1
-15
lines changed

sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala

Lines changed: 1 addition & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -40,22 +40,8 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext:
4040
private lazy val sparkSqlOperationManager = new SparkSQLOperationManager()
4141

4242
override def init(hiveConf: HiveConf) {
43-
setSuperField(this, "hiveConf", hiveConf)
44-
45-
// Create operation log root directory, if operation logging is enabled
46-
if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) {
47-
invoke(classOf[SessionManager], this, "initOperationLogRootDir")
48-
}
49-
50-
val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS)
51-
setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize))
52-
getAncestorField[Log](this, 3, "LOG").info(
53-
s"HiveServer2: Async execution pool size $backgroundPoolSize")
54-
5543
setSuperField(this, "operationManager", sparkSqlOperationManager)
56-
addService(sparkSqlOperationManager)
57-
58-
initCompositeService(hiveConf)
44+
super.init(hiveConf)
5945
}
6046

6147
override def openSession(

0 commit comments

Comments
 (0)