Skip to content

Commit

Permalink
[SPARK-7950] [SQL] Sets spark.sql.hive.version in HiveThriftServer2.s…
Browse files Browse the repository at this point in the history
…tartWithContext()

When starting `HiveThriftServer2` via `startWithContext`, property `spark.sql.hive.version` isn't set. This causes Simba ODBC driver 1.0.8.1006 behaves differently and fails simple queries.

Hive2 JDBC driver works fine in this case. Also, when starting the server with `start-thriftserver.sh`, both Hive2 JDBC driver and Simba ODBC driver works fine.

Please refer to [SPARK-7950] [1] for details.

[1]: https://issues.apache.org/jira/browse/SPARK-7950

Author: Cheng Lian <[email protected]>

Closes #6500 from liancheng/odbc-bugfix and squashes the following commits:

051e3a3 [Cheng Lian] Fixes import order
3a97376 [Cheng Lian] Sets spark.sql.hive.version in HiveThriftServer2.startWithContext()

(cherry picked from commit e7b6177)
Signed-off-by: Yin Huai <[email protected]>
  • Loading branch information
liancheng authored and yhuai committed May 29, 2015
1 parent 23bd05f commit caea7a6
Showing 1 changed file with 8 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,23 +17,23 @@

package org.apache.spark.sql.hive.thriftserver

import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer

import org.apache.commons.logging.LogFactory
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService}
import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor}
import org.apache.spark.sql.SQLConf

import org.apache.spark.{SparkContext, SparkConf, Logging}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart}
import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListenerApplicationEnd, SparkListener}
import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab
import org.apache.spark.sql.hive.{HiveContext, HiveShim}
import org.apache.spark.util.Utils

import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.{Logging, SparkContext}

/**
* The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a
Expand All @@ -51,6 +51,7 @@ object HiveThriftServer2 extends Logging {
@DeveloperApi
def startWithContext(sqlContext: HiveContext): Unit = {
val server = new HiveThriftServer2(sqlContext)
sqlContext.setConf("spark.sql.hive.version", HiveShim.version)
server.init(sqlContext.hiveconf)
server.start()
listener = new HiveThriftServer2Listener(server, sqlContext.conf)
Expand Down

0 comments on commit caea7a6

Please sign in to comment.