diff options
author | Cheng Lian <lian@databricks.com> | 2015-11-17 11:17:52 -0800 |
---|---|---|
committer | Michael Armbrust <michael@databricks.com> | 2015-11-17 11:17:52 -0800 |
commit | 7b1407c7b95c43299a30e891748824c4bc47e43b (patch) | |
tree | 5fbeef381ca979fc7a4ce3deec9f5c69d6d6c2cc /sql/hive-thriftserver/src/main | |
parent | e8833dd12c71b23a242727e86684d2d868ff84b3 (diff) | |
download | spark-7b1407c7b95c43299a30e891748824c4bc47e43b.tar.gz spark-7b1407c7b95c43299a30e891748824c4bc47e43b.tar.bz2 spark-7b1407c7b95c43299a30e891748824c4bc47e43b.zip |
[SPARK-11089][SQL] Adds option for disabling multi-session in Thrift server
This PR adds a new option `spark.sql.hive.thriftServer.singleSession` for disabling multi-session support in the Thrift server.
Note that this option is added as a Spark configuration (retrieved from `SparkConf`) rather than Spark SQL configuration (retrieved from `SQLConf`). This is because all SQL configurations are session-ized. Since multi-session support is by default on, no JDBC connection can modify global configurations like the newly added one.
Author: Cheng Lian <lian@databricks.com>
Closes #9740 from liancheng/spark-11089.single-session-option.
Diffstat (limited to 'sql/hive-thriftserver/src/main')
-rw-r--r-- | sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala | 6 |
1 files changed, 5 insertions, 1 deletions
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 33aaead3fb..af4fcdf021 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -66,7 +66,11 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext: val session = super.getSession(sessionHandle) HiveThriftServer2.listener.onSessionCreated( session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) - val ctx = hiveContext.newSession() + val ctx = if (hiveContext.hiveThriftServerSingleSession) { + hiveContext + } else { + hiveContext.newSession() + } ctx.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) sparkSqlOperationManager.sessionToContexts += sessionHandle -> ctx sessionHandle |