diff options
author | Cheng Lian <lian@databricks.com> | 2014-11-01 15:03:11 -0700 |
---|---|---|
committer | Michael Armbrust <michael@databricks.com> | 2014-11-01 15:03:11 -0700 |
commit | ad0fde10b2285e780349be5a8f333db0974a502f (patch) | |
tree | 4b870113fdb6d47ae4405edd3b8bd5d65d933e49 /sql/hive-thriftserver/src/main | |
parent | f55218aeb1e9d638df6229b36a59a15ce5363482 (diff) | |
download | spark-ad0fde10b2285e780349be5a8f333db0974a502f.tar.gz spark-ad0fde10b2285e780349be5a8f333db0974a502f.tar.bz2 spark-ad0fde10b2285e780349be5a8f333db0974a502f.zip |
[SPARK-4037][SQL] Removes the SessionState instance created in HiveThriftServer2
`HiveThriftServer2` creates a global singleton `SessionState` instance and overrides `HiveContext` to inject the `SessionState` object. This messes up `SessionState` initialization and causes problems.
This PR replaces the global `SessionState` with `HiveContext.sessionState` to avoid the initialization conflict. Also `HiveContext` reuses existing started `SessionState` if any (this is required by `SparkSQLCLIDriver`, which uses specialized `CliSessionState`).
Author: Cheng Lian <lian@databricks.com>
Closes #2887 from liancheng/spark-4037 and squashes the following commits:
8446675 [Cheng Lian] Removes redundant Driver initialization
a28fef5 [Cheng Lian] Avoid starting HiveContext.sessionState multiple times
49b1c5b [Cheng Lian] Reuses existing started SessionState if any
3cd6fab [Cheng Lian] Fixes SPARK-4037
Diffstat (limited to 'sql/hive-thriftserver/src/main')
2 files changed, 8 insertions, 27 deletions
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 3d468d8046..bd4e99492b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -17,11 +17,8 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ - import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} @@ -51,24 +48,12 @@ object HiveThriftServer2 extends Logging { def main(args: Array[String]) { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") - if (!optionsProcessor.process(args)) { System.exit(-1) } - val ss = new SessionState(new HiveConf(classOf[SessionState])) - - // Set all properties specified via command line. - val hiveConf: HiveConf = ss.getConf - hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logDebug(s"HiveConf var: $k=$v") - } - - SessionState.start(ss) - logInfo("Starting SparkContext") SparkSQLEnv.init() - SessionState.start(ss) Runtime.getRuntime.addShutdownHook( new Thread() { @@ -80,7 +65,7 @@ object HiveThriftServer2 extends Logging { try { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(hiveConf) + server.init(SparkSQLEnv.hiveContext.hiveconf) server.start() logInfo("HiveThriftServer2 started") } catch { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 2136a2ea63..5042586351 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.hive.thriftserver -import org.apache.hadoop.hive.ql.session.SessionState - -import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.Logging +import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} +import scala.collection.JavaConversions._ /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { @@ -37,14 +35,12 @@ private[hive] object SparkSQLEnv extends Logging { .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) sparkContext.addSparkListener(new StatsReportListener()) + hiveContext = new HiveContext(sparkContext) - hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = { - val state = SessionState.get() - setConf(state.getConf.getAllProperties) - state + if (log.isDebugEnabled) { + hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) => + logDebug(s"HiveConf var: $k=$v") } - @transient override lazy val hiveconf = sessionState.getConf } } } |