[SPARK-13076][SQL] Rename ClientInterface -> HiveClient

And ClientWrapper -> HiveClientImpl.

I have some followup pull requests to introduce a new internal catalog, and I think this new naming reflects better the functionality of the two classes.

Author: Reynold Xin <rxin@databricks.com>

Closes #10981 from rxin/SPARK-13076.
This commit is contained in:
Reynold Xin 2016-01-29 16:57:34 -08:00
parent e38b0baa38
commit 2cbc412821
12 changed files with 41 additions and 42 deletions

View file

@ -404,7 +404,7 @@ case class DescribeFunction(
result
}
case None => Seq(Row(s"Function: $functionName is not found."))
case None => Seq(Row(s"Function: $functionName not found."))
}
}
}

View file

@ -84,7 +84,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
"Extended Usage")
checkExistence(sql("describe functioN abcadf"), true,
"Function: abcadf is not found.")
"Function: abcadf not found.")
}
test("SPARK-6743: no columns from cache") {

View file

@ -79,8 +79,8 @@ class HiveContext private[hive](
sc: SparkContext,
cacheManager: CacheManager,
listener: SQLListener,
@transient private val execHive: ClientWrapper,
@transient private val metaHive: ClientInterface,
@transient private val execHive: HiveClientImpl,
@transient private val metaHive: HiveClient,
isRootContext: Boolean)
extends SQLContext(sc, cacheManager, listener, isRootContext) with Logging {
self =>
@ -193,7 +193,7 @@ class HiveContext private[hive](
* for storing persistent metadata, and only point to a dummy metastore in a temporary directory.
*/
@transient
protected[hive] lazy val executionHive: ClientWrapper = if (execHive != null) {
protected[hive] lazy val executionHive: HiveClientImpl = if (execHive != null) {
execHive
} else {
logInfo(s"Initializing execution hive, version $hiveExecutionVersion")
@ -203,7 +203,7 @@ class HiveContext private[hive](
config = newTemporaryConfiguration(useInMemoryDerby = true),
isolationOn = false,
baseClassLoader = Utils.getContextOrSparkClassLoader)
loader.createClient().asInstanceOf[ClientWrapper]
loader.createClient().asInstanceOf[HiveClientImpl]
}
/**
@ -222,7 +222,7 @@ class HiveContext private[hive](
* in the hive-site.xml file.
*/
@transient
protected[hive] lazy val metadataHive: ClientInterface = if (metaHive != null) {
protected[hive] lazy val metadataHive: HiveClient = if (metaHive != null) {
metaHive
} else {
val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion)

View file

@ -96,7 +96,7 @@ private[hive] object HiveSerDe {
}
}
private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: HiveContext)
private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveContext)
extends Catalog with Logging {
val conf = hive.conf

View file

@ -60,9 +60,9 @@ private[hive] case class HiveTable(
viewText: Option[String] = None) {
@transient
private[client] var client: ClientInterface = _
private[client] var client: HiveClient = _
private[client] def withClient(ci: ClientInterface): this.type = {
private[client] def withClient(ci: HiveClient): this.type = {
client = ci
this
}
@ -85,7 +85,7 @@ private[hive] case class HiveTable(
* internal and external classloaders for a given version of Hive and thus must expose only
* shared classes.
*/
private[hive] trait ClientInterface {
private[hive] trait HiveClient {
/** Returns the Hive Version of this client. */
def version: HiveVersion
@ -184,8 +184,8 @@ private[hive] trait ClientInterface {
/** Add a jar into class loader */
def addJar(path: String): Unit
/** Return a ClientInterface as new session, that will share the class loader and Hive client */
def newSession(): ClientInterface
/** Return a [[HiveClient]] as new session, that will share the class loader and Hive client */
def newSession(): HiveClient
/** Run a function within Hive state (SessionState, HiveConf, Hive client and class loader) */
def withHiveState[A](f: => A): A

View file

@ -44,8 +44,8 @@ import org.apache.spark.util.{CircularBuffer, Utils}
* A class that wraps the HiveClient and converts its responses to externally visible classes.
* Note that this class is typically loaded with an internal classloader for each instantiation,
* allowing it to interact directly with a specific isolated version of Hive. Loading this class
* with the isolated classloader however will result in it only being visible as a ClientInterface,
* not a ClientWrapper.
* with the isolated classloader however will result in it only being visible as a [[HiveClient]],
* not a [[HiveClientImpl]].
*
* This class needs to interact with multiple versions of Hive, but will always be compiled with
* the 'native', execution version of Hive. Therefore, any places where hive breaks compatibility
@ -55,14 +55,14 @@ import org.apache.spark.util.{CircularBuffer, Utils}
* @param config a collection of configuration options that will be added to the hive conf before
* opening the hive client.
* @param initClassLoader the classloader used when creating the `state` field of
* this ClientWrapper.
* this [[HiveClientImpl]].
*/
private[hive] class ClientWrapper(
private[hive] class HiveClientImpl(
override val version: HiveVersion,
config: Map[String, String],
initClassLoader: ClassLoader,
val clientLoader: IsolatedClientLoader)
extends ClientInterface
extends HiveClient
with Logging {
// Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur.
@ -77,7 +77,7 @@ private[hive] class ClientWrapper(
case hive.v1_2 => new Shim_v1_2()
}
// Create an internal session state for this ClientWrapper.
// Create an internal session state for this HiveClientImpl.
val state = {
val original = Thread.currentThread().getContextClassLoader
// Switch to the initClassLoader.
@ -160,7 +160,7 @@ private[hive] class ClientWrapper(
case e: Exception if causedByThrift(e) =>
caughtException = e
logWarning(
"HiveClientWrapper got thrift exception, destroying client and retrying " +
"HiveClient got thrift exception, destroying client and retrying " +
s"(${retryLimit - numTries} tries remaining)", e)
clientLoader.cachedHive = null
Thread.sleep(retryDelayMillis)
@ -199,7 +199,7 @@ private[hive] class ClientWrapper(
*/
def withHiveState[A](f: => A): A = retryLocked {
val original = Thread.currentThread().getContextClassLoader
// Set the thread local metastore client to the client associated with this ClientWrapper.
// Set the thread local metastore client to the client associated with this HiveClientImpl.
Hive.set(client)
// The classloader in clientLoader could be changed after addJar, always use the latest
// classloader
@ -521,8 +521,8 @@ private[hive] class ClientWrapper(
runSqlHive(s"ADD JAR $path")
}
def newSession(): ClientWrapper = {
clientLoader.createClient().asInstanceOf[ClientWrapper]
def newSession(): HiveClientImpl = {
clientLoader.createClient().asInstanceOf[HiveClientImpl]
}
def reset(): Unit = withHiveState {

View file

@ -38,8 +38,8 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.types.{IntegralType, StringType}
/**
* A shim that defines the interface between ClientWrapper and the underlying Hive library used to
* talk to the metastore. Each Hive version has its own implementation of this class, defining
* A shim that defines the interface between [[HiveClientImpl]] and the underlying Hive library used
* to talk to the metastore. Each Hive version has its own implementation of this class, defining
* version-specific version of needed functions.
*
* The guideline for writing shims is:
@ -52,7 +52,6 @@ private[client] sealed abstract class Shim {
/**
* Set the current SessionState to the given SessionState. Also, set the context classloader of
* the current thread to the one set in the HiveConf of this given `state`.
* @param state
*/
def setCurrentSessionState(state: SessionState): Unit

View file

@ -124,15 +124,15 @@ private[hive] object IsolatedClientLoader extends Logging {
}
/**
* Creates a Hive `ClientInterface` using a classloader that works according to the following rules:
* Creates a [[HiveClient]] using a classloader that works according to the following rules:
* - Shared classes: Java, Scala, logging, and Spark classes are delegated to `baseClassLoader`
* allowing the results of calls to the `ClientInterface` to be visible externally.
* allowing the results of calls to the [[HiveClient]] to be visible externally.
* - Hive classes: new instances are loaded from `execJars`. These classes are not
* accessible externally due to their custom loading.
* - ClientWrapper: a new copy is created for each instance of `IsolatedClassLoader`.
* - [[HiveClientImpl]]: a new copy is created for each instance of `IsolatedClassLoader`.
* This new instance is able to see a specific version of hive without using reflection. Since
* this is a unique instance, it is not visible externally other than as a generic
* `ClientInterface`, unless `isolationOn` is set to `false`.
* [[HiveClient]], unless `isolationOn` is set to `false`.
*
* @param version The version of hive on the classpath. used to pick specific function signatures
* that are not compatible across versions.
@ -179,7 +179,7 @@ private[hive] class IsolatedClientLoader(
/** True if `name` refers to a spark class that must see specific version of Hive. */
protected def isBarrierClass(name: String): Boolean =
name.startsWith(classOf[ClientWrapper].getName) ||
name.startsWith(classOf[HiveClientImpl].getName) ||
name.startsWith(classOf[Shim].getName) ||
barrierPrefixes.exists(name.startsWith)
@ -233,9 +233,9 @@ private[hive] class IsolatedClientLoader(
}
/** The isolated client interface to Hive. */
private[hive] def createClient(): ClientInterface = {
private[hive] def createClient(): HiveClient = {
if (!isolationOn) {
return new ClientWrapper(version, config, baseClassLoader, this)
return new HiveClientImpl(version, config, baseClassLoader, this)
}
// Pre-reflective instantiation setup.
logDebug("Initializing the logger to avoid disaster...")
@ -244,10 +244,10 @@ private[hive] class IsolatedClientLoader(
try {
classLoader
.loadClass(classOf[ClientWrapper].getName)
.loadClass(classOf[HiveClientImpl].getName)
.getConstructors.head
.newInstance(version, config, classLoader, this)
.asInstanceOf[ClientInterface]
.asInstanceOf[HiveClient]
} catch {
case e: InvocationTargetException =>
if (e.getCause().isInstanceOf[NoClassDefFoundError]) {

View file

@ -38,13 +38,13 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.catalyst.util.sequenceOption
import org.apache.spark.sql.hive.HiveShim._
import org.apache.spark.sql.hive.client.ClientWrapper
import org.apache.spark.sql.hive.client.HiveClientImpl
import org.apache.spark.sql.types._
private[hive] class HiveFunctionRegistry(
underlying: analysis.FunctionRegistry,
executionHive: ClientWrapper)
executionHive: HiveClientImpl)
extends analysis.FunctionRegistry with HiveInspectors {
def getFunctionInfo(name: String): FunctionInfo = {

View file

@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.CacheTableCommand
import org.apache.spark.sql.hive._
import org.apache.spark.sql.hive.client.ClientWrapper
import org.apache.spark.sql.hive.client.HiveClientImpl
import org.apache.spark.sql.hive.execution.HiveNativeCommand
import org.apache.spark.util.{ShutdownHookManager, Utils}
@ -458,7 +458,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
org.apache.spark.sql.catalyst.analysis.FunctionRegistry.builtin.copy(), this.executionHive)
}
private[hive] class TestHiveFunctionRegistry(fr: SimpleFunctionRegistry, client: ClientWrapper)
private[hive] class TestHiveFunctionRegistry(fr: SimpleFunctionRegistry, client: HiveClientImpl)
extends HiveFunctionRegistry(fr, client) {
private val removedFunctions =

View file

@ -30,7 +30,7 @@ import org.apache.spark.tags.ExtendedHiveTest
import org.apache.spark.util.Utils
/**
* A simple set of tests that call the methods of a hive ClientInterface, loading different version
* A simple set of tests that call the methods of a [[HiveClient]], loading different version
* of hive from maven central. These tests are simple in that they are mostly just testing to make
* sure that reflective calls are not throwing NoSuchMethod error, but the actually functionality
* is not fully tested.
@ -101,7 +101,7 @@ class VersionsSuite extends SparkFunSuite with Logging {
private val versions = Seq("12", "13", "14", "1.0.0", "1.1.0", "1.2.0")
private var client: ClientInterface = null
private var client: HiveClient = null
versions.foreach { version =>
test(s"$version: create client") {

View file

@ -199,7 +199,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
"Extended Usage")
checkExistence(sql("describe functioN abcadf"), true,
"Function: abcadf is not found.")
"Function: abcadf not found.")
checkExistence(sql("describe functioN `~`"), true,
"Function: ~",