Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,6 @@

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
Expand All @@ -29,12 +26,15 @@ import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart}
import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab
import org.apache.spark.sql.hive.{HiveContext, HiveShim}
import org.apache.spark.util.Utils
import org.apache.spark.{Logging, SparkContext}

import scala.collection.mutable
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

note: import out of order. i will fix it in this case.

import scala.collection.mutable.ArrayBuffer

/**
* The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a
* `HiveThriftServer2` thrift server.
Expand All @@ -51,7 +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)
sqlContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion)
server.init(sqlContext.hiveconf)
server.start()
listener = new HiveThriftServer2Listener(server, sqlContext.conf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,66 +18,31 @@
package org.apache.spark.sql.hive.thriftserver

import java.sql.{Date, Timestamp}
import java.util.concurrent.Executors
import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, UUID}

import org.apache.commons.logging.Log
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hive.service.cli.thrift.TProtocolVersion
import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager

import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, Map => SMap}
import java.util.{Map => JMap, UUID}

import org.apache.hadoop.hive.metastore.api.FieldSchema
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hive.service.cli._
import org.apache.hive.service.cli.operation.ExecuteStatementOperation
import org.apache.hive.service.cli.session.{SessionManager, HiveSession}
import org.apache.hive.service.cli.session.HiveSession

import org.apache.spark.{SparkContext, Logging}
import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf}
import org.apache.spark.Logging
import org.apache.spark.sql.execution.SetCommand
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
import org.apache.spark.sql.types._
import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf}

/**
* A compatibility layer for interacting with Hive version 0.13.1.
*/
private[thriftserver] object HiveThriftServerShim {
val version = "0.13.1"

def setServerUserName(
sparkServiceUGI: UserGroupInformation,
sparkCliService:SparkSQLCLIService) = {
setSuperField(sparkCliService, "serviceUGI", sparkServiceUGI)
}
}

private[hive] class SparkSQLDriver(val _context: HiveContext = SparkSQLEnv.hiveContext)
extends AbstractSparkSQLDriver(_context) {
override def getResults(res: JList[_]): Boolean = {
if (hiveResponse == null) {
false
} else {
res.asInstanceOf[JArrayList[String]].addAll(hiveResponse)
hiveResponse = null
true
}
}
}
import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, Map => SMap}

private[hive] class SparkExecuteStatementOperation(
parentSession: HiveSession,
statement: String,
confOverlay: JMap[String, String],
runInBackground: Boolean = true)(
hiveContext: HiveContext,
sessionToActivePool: SMap[SessionHandle, String])
runInBackground: Boolean = true)
(hiveContext: HiveContext, sessionToActivePool: SMap[SessionHandle, String])
// NOTE: `runInBackground` is set to `false` intentionally to disable asynchronous execution
extends ExecuteStatementOperation(parentSession, statement, confOverlay, false) with Logging {
extends ExecuteStatementOperation(parentSession, statement, confOverlay, false)
with Logging {

private var result: DataFrame = _
private var iter: Iterator[SparkRow] = _
Expand All @@ -88,7 +53,7 @@ private[hive] class SparkExecuteStatementOperation(
logDebug("CLOSING")
}

def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) {
def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) {
dataTypes(ordinal) match {
case StringType =>
to += from.getString(ordinal)
Expand Down Expand Up @@ -209,48 +174,3 @@ private[hive] class SparkExecuteStatementOperation(
HiveThriftServer2.listener.onStatementFinish(statementId)
}
}

private[hive] class SparkSQLSessionManager(hiveContext: HiveContext)
extends SessionManager
with ReflectedCompositeService {

private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext)

override def init(hiveConf: HiveConf) {
setSuperField(this, "hiveConf", hiveConf)

val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS)
setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize))
getAncestorField[Log](this, 3, "LOG").info(
s"HiveServer2: Async execution pool size $backgroundPoolSize")

setSuperField(this, "operationManager", sparkSqlOperationManager)
addService(sparkSqlOperationManager)

initCompositeService(hiveConf)
}

override def openSession(
protocol: TProtocolVersion,
username: String,
passwd: String,
sessionConf: java.util.Map[String, String],
withImpersonation: Boolean,
delegationToken: String): SessionHandle = {
hiveContext.openSession()
val sessionHandle = super.openSession(
protocol, username, passwd, sessionConf, withImpersonation, delegationToken)
val session = super.getSession(sessionHandle)
HiveThriftServer2.listener.onSessionCreated(
session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername)
sessionHandle
}

override def closeSession(sessionHandle: SessionHandle) {
HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString)
super.closeSession(sessionHandle)
sparkSqlOperationManager.sessionToActivePool -= sessionHandle

hiveContext.detachSession()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,12 +32,12 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils}
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.exec.Utilities
import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor}
import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor, CommandProcessorFactory}
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.thrift.transport.TSocket

import org.apache.spark.Logging
import org.apache.spark.sql.hive.{HiveContext, HiveShim}
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.util.Utils

private[hive] object SparkSQLCLIDriver {
Expand Down Expand Up @@ -267,7 +267,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
} else {
var ret = 0
val hconf = conf.asInstanceOf[HiveConf]
val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf)
val proc: CommandProcessor = CommandProcessorFactory.get(Array(tokens(0)), hconf)

if (proc != null) {
if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] ||
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@ import java.io.IOException
import java.util.{List => JList}
import javax.security.auth.login.LoginException

import scala.collection.JavaConversions._

import org.apache.commons.logging.Log
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.shims.ShimLoader
Expand All @@ -34,7 +32,8 @@ import org.apache.hive.service.{AbstractService, Service, ServiceException}

import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
import org.apache.spark.util.Utils

import scala.collection.JavaConversions._

private[hive] class SparkSQLCLIService(hiveContext: HiveContext)
extends CLIService
Expand All @@ -52,7 +51,7 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext)
try {
HiveAuthFactory.loginFromKeytab(hiveConf)
sparkServiceUGI = ShimLoader.getHadoopShims.getUGIForConf(hiveConf)
HiveThriftServerShim.setServerUserName(sparkServiceUGI, this)
setSuperField(this, "serviceUGI", sparkServiceUGI)
} catch {
case e @ (_: IOException | _: LoginException) =>
throw new ServiceException("Unable to login to kerberos with given principal/keytab", e)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

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

import scala.collection.JavaConversions._
import java.util.{ArrayList => JArrayList, List => JList}

import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema}
Expand All @@ -27,8 +27,12 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse
import org.apache.spark.Logging
import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}

private[hive] abstract class AbstractSparkSQLDriver(
val context: HiveContext = SparkSQLEnv.hiveContext) extends Driver with Logging {
import scala.collection.JavaConversions._

private[hive] class SparkSQLDriver(
val context: HiveContext = SparkSQLEnv.hiveContext)
extends Driver
with Logging {

private[hive] var tableSchema: Schema = _
private[hive] var hiveResponse: Seq[String] = _
Expand Down Expand Up @@ -71,6 +75,16 @@ private[hive] abstract class AbstractSparkSQLDriver(
0
}

override def getResults(res: JList[_]): Boolean = {
if (hiveResponse == null) {
false
} else {
res.asInstanceOf[JArrayList[String]].addAll(hiveResponse)
hiveResponse = null
true
}
}

override def getSchema: Schema = tableSchema

override def destroy() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.io.PrintStream
import scala.collection.JavaConversions._

import org.apache.spark.scheduler.StatsReportListener
import org.apache.spark.sql.hive.{HiveShim, HiveContext}
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.{Logging, SparkConf, SparkContext}
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -56,7 +56,7 @@ private[hive] object SparkSQLEnv extends Logging {
hiveContext.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8"))
hiveContext.metadataHive.setError(new PrintStream(System.err, true, "UTF-8"))

hiveContext.setConf("spark.sql.hive.version", HiveShim.version)
hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion)

if (log.isDebugEnabled) {
hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) =>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

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

import java.util.concurrent.Executors

import org.apache.commons.logging.Log
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hive.service.cli.SessionHandle
import org.apache.hive.service.cli.session.SessionManager
import org.apache.hive.service.cli.thrift.TProtocolVersion

import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager

private[hive] class SparkSQLSessionManager(hiveContext: HiveContext)
extends SessionManager
with ReflectedCompositeService {

private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext)

override def init(hiveConf: HiveConf) {
setSuperField(this, "hiveConf", hiveConf)

val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS)
setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize))
getAncestorField[Log](this, 3, "LOG").info(
s"HiveServer2: Async execution pool size $backgroundPoolSize")

setSuperField(this, "operationManager", sparkSqlOperationManager)
addService(sparkSqlOperationManager)

initCompositeService(hiveConf)
}

override def openSession(protocol: TProtocolVersion,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

note for the future: we should indent this 4 space. i will fix it during merge.

username: String,
passwd: String,
sessionConf: java.util.Map[String, String],
withImpersonation: Boolean,
delegationToken: String): SessionHandle = {
hiveContext.openSession()
val sessionHandle = super.openSession(
protocol, username, passwd, sessionConf, withImpersonation, delegationToken)
val session = super.getSession(sessionHandle)
HiveThriftServer2.listener.onSessionCreated(
session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername)
sessionHandle
}

override def closeSession(sessionHandle: SessionHandle) {
HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString)
super.closeSession(sessionHandle)
sparkSqlOperationManager.sessionToActivePool -= sessionHandle

hiveContext.detachSession()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.thrift.transport.TSocket
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.{Logging, SparkFunSuite}
import org.apache.spark.sql.hive.HiveShim
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.util.Utils

object TestData {
Expand Down Expand Up @@ -111,7 +111,8 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest {
withJdbcStatement { statement =>
val resultSet = statement.executeQuery("SET spark.sql.hive.version")
resultSet.next()
assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}")
assert(resultSet.getString(1) ===
s"spark.sql.hive.version=${HiveContext.hiveExecutionVersion}")
}
}

Expand Down Expand Up @@ -365,7 +366,8 @@ class HiveThriftHttpServerSuite extends HiveThriftJdbcTest {
withJdbcStatement { statement =>
val resultSet = statement.executeQuery("SET spark.sql.hive.version")
resultSet.next()
assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}")
assert(resultSet.getString(1) ===
s"spark.sql.hive.version=${HiveContext.hiveExecutionVersion}")
}
}
}
Expand Down
Loading