You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2016/04/20 21:58:57 UTC

[2/2] spark git commit: [SPARK-14720][SPARK-13643] Move Hive-specific methods into HiveSessionState and Create a SparkSession class

[SPARK-14720][SPARK-13643] Move Hive-specific methods into HiveSessionState and Create a SparkSession class

## What changes were proposed in this pull request?
This PR has two main changes.
1. Move Hive-specific methods from HiveContext to HiveSessionState, which help the work of removing HiveContext.
2. Create a SparkSession Class, which will later be the entry point of Spark SQL users.

## How was this patch tested?
Existing tests

This PR is trying to fix test failures of https://github.com/apache/spark/pull/12485.

Author: Andrew Or <an...@databricks.com>
Author: Yin Huai <yh...@databricks.com>

Closes #12522 from yhuai/spark-session.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8fc267ab
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8fc267ab
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8fc267ab

Branch: refs/heads/master
Commit: 8fc267ab3322e46db81e725a5cb1adb5a71b2b4d
Parents: cb8ea9e
Author: Andrew Or <an...@databricks.com>
Authored: Wed Apr 20 12:58:48 2016 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Wed Apr 20 12:58:48 2016 -0700

----------------------------------------------------------------------
 .../apache/spark/internal/config/package.scala  |   7 +
 .../scala/org/apache/spark/sql/SQLContext.scala |  52 ++--
 .../org/apache/spark/sql/SparkSession.scala     | 100 +++++++
 .../spark/sql/internal/SessionState.scala       |  54 +++-
 .../SparkExecuteStatementOperation.scala        |   2 +-
 .../sql/hive/thriftserver/SparkSQLDriver.scala  |   7 +-
 .../sql/hive/thriftserver/SparkSQLEnv.scala     |   6 +-
 .../thriftserver/SparkSQLSessionManager.scala   |   2 +-
 .../server/SparkSQLOperationManager.scala       |   2 +-
 .../hive/execution/HiveCompatibilitySuite.scala |   6 +-
 .../HiveWindowFunctionQuerySuite.scala          |  31 ++-
 .../org/apache/spark/sql/hive/HiveContext.scala | 221 ++-------------
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  30 +-
 .../spark/sql/hive/HiveQueryExecution.scala     |  66 +++++
 .../spark/sql/hive/HiveSessionCatalog.scala     |  11 +-
 .../spark/sql/hive/HiveSessionState.scala       | 180 ++++++++++--
 .../apache/spark/sql/hive/HiveStrategies.scala  |  11 +-
 .../org/apache/spark/sql/hive/TableReader.scala |  17 +-
 .../spark/sql/hive/client/HiveClientImpl.scala  |   2 +
 .../hive/execution/CreateTableAsSelect.scala    |   9 +-
 .../sql/hive/execution/CreateViewAsSelect.scala |  11 +-
 .../sql/hive/execution/HiveNativeCommand.scala  |   8 +-
 .../sql/hive/execution/HiveSqlParser.scala      |  19 +-
 .../sql/hive/execution/HiveTableScan.scala      |   6 +-
 .../hive/execution/InsertIntoHiveTable.scala    |  23 +-
 .../hive/execution/ScriptTransformation.scala   |   9 +-
 .../spark/sql/hive/execution/commands.scala     |  38 ++-
 .../apache/spark/sql/hive/test/TestHive.scala   | 275 +++++++++++--------
 .../spark/sql/hive/ErrorPositionSuite.scala     |   3 +-
 .../spark/sql/hive/HiveContextSuite.scala       |   7 +-
 .../sql/hive/HiveMetastoreCatalogSuite.scala    |   6 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala    |  33 ++-
 .../spark/sql/hive/MultiDatabaseSuite.scala     |   9 +-
 .../apache/spark/sql/hive/StatisticsSuite.scala |   4 +-
 .../hive/execution/BigDataBenchmarkSuite.scala  |   4 +-
 .../sql/hive/execution/HiveComparisonTest.scala |  25 +-
 .../sql/hive/execution/HiveQuerySuite.scala     |   6 +-
 .../sql/hive/execution/HiveSerDeSuite.scala     |   4 +-
 .../spark/sql/hive/execution/PruningSuite.scala |  18 +-
 .../sql/hive/execution/SQLQuerySuite.scala      |   4 +-
 .../execution/ScriptTransformationSuite.scala   |   8 +-
 .../apache/spark/sql/hive/parquetSuites.scala   |   6 +-
 .../spark/sql/sources/BucketedReadSuite.scala   |   2 +-
 43 files changed, 797 insertions(+), 547 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/core/src/main/scala/org/apache/spark/internal/config/package.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 94b50ee..2c1e0b7 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -89,4 +89,11 @@ package object config {
     .stringConf
     .toSequence
     .createWithDefault(Nil)
+
+  // Note: This is a SQL config but needs to be in core because the REPL depends on it
+  private[spark] val CATALOG_IMPLEMENTATION = ConfigBuilder("spark.sql.catalogImplementation")
+    .internal()
+    .stringConf
+    .checkValues(Set("hive", "in-memory"))
+    .createWithDefault("in-memory")
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 781d699..f3f8414 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -63,14 +63,18 @@ import org.apache.spark.util.Utils
  * @since 1.0.0
  */
 class SQLContext private[sql](
-    @transient protected[sql] val sharedState: SharedState,
+    @transient private val sparkSession: SparkSession,
     val isRootContext: Boolean)
   extends Logging with Serializable {
 
   self =>
 
+  private[sql] def this(sparkSession: SparkSession) = {
+    this(sparkSession, true)
+  }
+
   def this(sc: SparkContext) = {
-    this(new SharedState(sc), true)
+    this(new SparkSession(sc))
   }
 
   def this(sparkContext: JavaSparkContext) = this(sparkContext.sc)
@@ -97,12 +101,15 @@ class SQLContext private[sql](
     }
   }
 
-  def sparkContext: SparkContext = sharedState.sparkContext
-
+  protected[sql] def sessionState: SessionState = sparkSession.sessionState
+  protected[sql] def sharedState: SharedState = sparkSession.sharedState
+  protected[sql] def conf: SQLConf = sessionState.conf
   protected[sql] def cacheManager: CacheManager = sharedState.cacheManager
   protected[sql] def listener: SQLListener = sharedState.listener
   protected[sql] def externalCatalog: ExternalCatalog = sharedState.externalCatalog
 
+  def sparkContext: SparkContext = sharedState.sparkContext
+
   /**
    * Returns a [[SQLContext]] as new session, with separated SQL configurations, temporary
    * tables, registered functions, but sharing the same [[SparkContext]], cached data and
@@ -110,14 +117,9 @@ class SQLContext private[sql](
    *
    * @since 1.6.0
    */
-  def newSession(): SQLContext = new SQLContext(sharedState, isRootContext = false)
-
-  /**
-   * Per-session state, e.g. configuration, functions, temporary tables etc.
-   */
-  @transient
-  protected[sql] lazy val sessionState: SessionState = new SessionState(self)
-  protected[spark] def conf: SQLConf = sessionState.conf
+  def newSession(): SQLContext = {
+    new SQLContext(sparkSession.newSession(), isRootContext = false)
+  }
 
   /**
    * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s
@@ -132,10 +134,14 @@ class SQLContext private[sql](
    * @group config
    * @since 1.0.0
    */
-  def setConf(props: Properties): Unit = conf.setConf(props)
+  def setConf(props: Properties): Unit = sessionState.setConf(props)
 
-  /** Set the given Spark SQL configuration property. */
-  private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = conf.setConf(entry, value)
+  /**
+   * Set the given Spark SQL configuration property.
+   */
+  private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = {
+    sessionState.setConf(entry, value)
+  }
 
   /**
    * Set the given Spark SQL configuration property.
@@ -143,7 +149,7 @@ class SQLContext private[sql](
    * @group config
    * @since 1.0.0
    */
-  def setConf(key: String, value: String): Unit = conf.setConfString(key, value)
+  def setConf(key: String, value: String): Unit = sessionState.setConf(key, value)
 
   /**
    * Return the value of Spark SQL configuration property for the given key.
@@ -186,23 +192,19 @@ class SQLContext private[sql](
    */
   def getAllConfs: immutable.Map[String, String] = conf.getAllConfs
 
-  // Extract `spark.sql.*` entries and put it in our SQLConf.
-  // Subclasses may additionally set these entries in other confs.
-  SQLContext.getSQLProperties(sparkContext.getConf).asScala.foreach { case (k, v) =>
-    setConf(k, v)
-  }
-
   protected[sql] def parseSql(sql: String): LogicalPlan = sessionState.sqlParser.parsePlan(sql)
 
   protected[sql] def executeSql(sql: String): QueryExecution = executePlan(parseSql(sql))
 
-  protected[sql] def executePlan(plan: LogicalPlan) = new QueryExecution(this, plan)
+  protected[sql] def executePlan(plan: LogicalPlan): QueryExecution = {
+    sessionState.executePlan(plan)
+  }
 
   /**
    * Add a jar to SQLContext
    */
   protected[sql] def addJar(path: String): Unit = {
-    sparkContext.addJar(path)
+    sessionState.addJar(path)
   }
 
   /** A [[FunctionResourceLoader]] that can be used in SessionCatalog. */
@@ -768,7 +770,7 @@ class SQLContext private[sql](
    * as Spark can parse all supported Hive DDLs itself.
    */
   private[sql] def runNativeSql(sqlText: String): Seq[Row] = {
-    throw new UnsupportedOperationException
+    sessionState.runNativeSql(sqlText).map { r => Row(r) }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
new file mode 100644
index 0000000..17ba299
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -0,0 +1,100 @@
+/*
+ * 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
+
+import scala.reflect.ClassTag
+import scala.util.control.NonFatal
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.util.Utils
+
+
+/**
+ * The entry point to Spark execution.
+ */
+class SparkSession private(
+    sparkContext: SparkContext,
+    existingSharedState: Option[SharedState]) { self =>
+
+  def this(sc: SparkContext) {
+    this(sc, None)
+  }
+
+  /**
+   * Start a new session where configurations, temp tables, temp functions etc. are isolated.
+   */
+  def newSession(): SparkSession = {
+    // Note: materialize the shared state here to ensure the parent and child sessions are
+    // initialized with the same shared state.
+    new SparkSession(sparkContext, Some(sharedState))
+  }
+
+  @transient
+  protected[sql] lazy val sharedState: SharedState = {
+    existingSharedState.getOrElse(
+      SparkSession.reflect[SharedState, SparkContext](
+        SparkSession.sharedStateClassName(sparkContext.conf),
+        sparkContext))
+  }
+
+  @transient
+  protected[sql] lazy val sessionState: SessionState = {
+    SparkSession.reflect[SessionState, SQLContext](
+      SparkSession.sessionStateClassName(sparkContext.conf),
+      new SQLContext(self, isRootContext = false))
+  }
+
+}
+
+
+private object SparkSession {
+
+  private def sharedStateClassName(conf: SparkConf): String = {
+    conf.get(CATALOG_IMPLEMENTATION) match {
+      case "hive" => "org.apache.spark.sql.hive.HiveSharedState"
+      case "in-memory" => classOf[SharedState].getCanonicalName
+    }
+  }
+
+  private def sessionStateClassName(conf: SparkConf): String = {
+    conf.get(CATALOG_IMPLEMENTATION) match {
+      case "hive" => "org.apache.spark.sql.hive.HiveSessionState"
+      case "in-memory" => classOf[SessionState].getCanonicalName
+    }
+  }
+
+  /**
+   * Helper method to create an instance of [[T]] using a single-arg constructor that
+   * accepts an [[Arg]].
+   */
+  private def reflect[T, Arg <: AnyRef](
+      className: String,
+      ctorArg: Arg)(implicit ctorArgTag: ClassTag[Arg]): T = {
+    try {
+      val clazz = Utils.classForName(className)
+      val ctor = clazz.getDeclaredConstructor(ctorArgTag.runtimeClass)
+      ctor.newInstance(ctorArg).asInstanceOf[T]
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException(s"Error while instantiating '$className':", e)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
index d404a7c..42915d5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
@@ -17,15 +17,22 @@
 
 package org.apache.spark.sql.internal
 
+import java.util.Properties
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.internal.config.ConfigEntry
 import org.apache.spark.sql.{ContinuousQueryManager, ExperimentalMethods, SQLContext, UDFRegistration}
 import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
 import org.apache.spark.sql.catalyst.catalog.SessionCatalog
 import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, PreInsertCastAndRename, ResolveDataSource}
 import org.apache.spark.sql.util.ExecutionListenerManager
 
+
 /**
  * A class that holds all session-specific state in a given [[SQLContext]].
  */
@@ -37,7 +44,10 @@ private[sql] class SessionState(ctx: SQLContext) {
   /**
    * SQL-specific key-value configurations.
    */
-  lazy val conf = new SQLConf
+  lazy val conf: SQLConf = new SQLConf
+
+  // Automatically extract `spark.sql.*` entries and put it in our SQLConf
+  setConf(SQLContext.getSQLProperties(ctx.sparkContext.getConf))
 
   lazy val experimentalMethods = new ExperimentalMethods
 
@@ -101,5 +111,45 @@ private[sql] class SessionState(ctx: SQLContext) {
    * Interface to start and stop [[org.apache.spark.sql.ContinuousQuery]]s.
    */
   lazy val continuousQueryManager: ContinuousQueryManager = new ContinuousQueryManager(ctx)
-}
 
+
+  // ------------------------------------------------------
+  //  Helper methods, partially leftover from pre-2.0 days
+  // ------------------------------------------------------
+
+  def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(ctx, plan)
+
+  def refreshTable(tableName: String): Unit = {
+    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
+  }
+
+  def invalidateTable(tableName: String): Unit = {
+    catalog.invalidateTable(sqlParser.parseTableIdentifier(tableName))
+  }
+
+  final def setConf(properties: Properties): Unit = {
+    properties.asScala.foreach { case (k, v) => setConf(k, v) }
+  }
+
+  final def setConf[T](entry: ConfigEntry[T], value: T): Unit = {
+    conf.setConf(entry, value)
+    setConf(entry.key, entry.stringConverter(value))
+  }
+
+  def setConf(key: String, value: String): Unit = {
+    conf.setConfString(key, value)
+  }
+
+  def addJar(path: String): Unit = {
+    ctx.sparkContext.addJar(path)
+  }
+
+  def analyze(tableName: String): Unit = {
+    throw new UnsupportedOperationException
+  }
+
+  def runNativeSql(sql: String): Seq[String] = {
+    throw new UnsupportedOperationException
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 673a293..d89c3b4 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -195,7 +195,7 @@ private[hive] class SparkExecuteStatementOperation(
     setState(OperationState.RUNNING)
     // Always use the latest class loader provided by executionHive's state.
     val executionHiveClassLoader =
-      hiveContext.executionHive.state.getConf.getClassLoader
+      hiveContext.sessionState.executionHive.state.getConf.getClassLoader
     Thread.currentThread().setContextClassLoader(executionHiveClassLoader)
 
     HiveThriftServer2.listener.onStatementStart(

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
index b8bc8ea..7e8eada 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
@@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse
 
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
+import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes, HiveQueryExecution}
 
 private[hive] class SparkSQLDriver(
     val context: HiveContext = SparkSQLEnv.hiveContext)
@@ -41,7 +41,7 @@ private[hive] class SparkSQLDriver(
   override def init(): Unit = {
   }
 
-  private def getResultSetSchema(query: context.QueryExecution): Schema = {
+  private def getResultSetSchema(query: HiveQueryExecution): Schema = {
     val analyzed = query.analyzed
     logDebug(s"Result Schema: ${analyzed.output}")
     if (analyzed.output.isEmpty) {
@@ -59,7 +59,8 @@ private[hive] class SparkSQLDriver(
     // TODO unify the error code
     try {
       context.sparkContext.setJobDescription(command)
-      val execution = context.executePlan(context.sql(command).logicalPlan)
+      val execution =
+        context.executePlan(context.sql(command).logicalPlan).asInstanceOf[HiveQueryExecution]
       hiveResponse = execution.stringResult()
       tableSchema = getResultSetSchema(execution)
       new CommandProcessorResponse(0)

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala
----------------------------------------------------------------------
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 ae1d737..2679ac1 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
@@ -58,9 +58,9 @@ private[hive] object SparkSQLEnv extends Logging {
       sparkContext.addSparkListener(new StatsReportListener())
       hiveContext = new HiveContext(sparkContext)
 
-      hiveContext.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8"))
-      hiveContext.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8"))
-      hiveContext.metadataHive.setError(new PrintStream(System.err, true, "UTF-8"))
+      hiveContext.sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8"))
+      hiveContext.sessionState.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8"))
+      hiveContext.sessionState.metadataHive.setError(new PrintStream(System.err, true, "UTF-8"))
 
       hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
----------------------------------------------------------------------
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 de4e9c6..f492b56 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
@@ -71,7 +71,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext:
     val session = super.getSession(sessionHandle)
     HiveThriftServer2.listener.onSessionCreated(
       session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername)
-    val ctx = if (hiveContext.hiveThriftServerSingleSession) {
+    val ctx = if (hiveContext.sessionState.hiveThriftServerSingleSession) {
       hiveContext
     } else {
       hiveContext.newSession()

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
index 0c468a4..da410c6 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
@@ -47,7 +47,7 @@ private[thriftserver] class SparkSQLOperationManager()
       confOverlay: JMap[String, String],
       async: Boolean): ExecuteStatementOperation = synchronized {
     val hiveContext = sessionToContexts(parentSession.getSessionHandle)
-    val runInBackground = async && hiveContext.hiveThriftServerAsync
+    val runInBackground = async && hiveContext.sessionState.hiveThriftServerAsync
     val operation = new SparkExecuteStatementOperation(parentSession, statement, confOverlay,
       runInBackground)(hiveContext, sessionToActivePool)
     handleToOperation.put(operation.getHandle, operation)

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 989e68a..49fd198 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -39,7 +39,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
   private val originalLocale = Locale.getDefault
   private val originalColumnBatchSize = TestHive.conf.columnBatchSize
   private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning
-  private val originalConvertMetastoreOrc = TestHive.convertMetastoreOrc
+  private val originalConvertMetastoreOrc = TestHive.sessionState.convertMetastoreOrc
 
   def testCases: Seq[(String, File)] = {
     hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f)
@@ -47,7 +47,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
 
   override def beforeAll() {
     super.beforeAll()
-    TestHive.cacheTables = true
+    TestHive.setCacheTables(true)
     // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
     TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
     // Add Locale setting
@@ -66,7 +66,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
 
   override def afterAll() {
     try {
-      TestHive.cacheTables = false
+      TestHive.setCacheTables(false)
       TimeZone.setDefault(originalTimeZone)
       Locale.setDefault(originalLocale)
       TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize)

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala
index d0b4cbe..de592f8 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala
@@ -38,7 +38,8 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte
   private val testTempDir = Utils.createTempDir()
 
   override def beforeAll() {
-    TestHive.cacheTables = true
+    super.beforeAll()
+    TestHive.setCacheTables(true)
     // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
     TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
     // Add Locale setting
@@ -100,11 +101,14 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte
   }
 
   override def afterAll() {
-    TestHive.cacheTables = false
-    TimeZone.setDefault(originalTimeZone)
-    Locale.setDefault(originalLocale)
-    TestHive.reset()
-    super.afterAll()
+    try {
+      TestHive.setCacheTables(false)
+      TimeZone.setDefault(originalTimeZone)
+      Locale.setDefault(originalLocale)
+      TestHive.reset()
+    } finally {
+      super.afterAll()
+    }
   }
 
   /////////////////////////////////////////////////////////////////////////////
@@ -773,7 +777,8 @@ class HiveWindowFunctionQueryFileSuite
   private val testTempDir = Utils.createTempDir()
 
   override def beforeAll() {
-    TestHive.cacheTables = true
+    super.beforeAll()
+    TestHive.setCacheTables(true)
     // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
     TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
     // Add Locale setting
@@ -790,10 +795,14 @@ class HiveWindowFunctionQueryFileSuite
   }
 
   override def afterAll() {
-    TestHive.cacheTables = false
-    TimeZone.setDefault(originalTimeZone)
-    Locale.setDefault(originalLocale)
-    TestHive.reset()
+    try {
+      TestHive.setCacheTables(false)
+      TimeZone.setDefault(originalTimeZone)
+      Locale.setDefault(originalLocale)
+      TestHive.reset()
+    } finally {
+      super.afterAll()
+    }
   }
 
   override def blackList: Seq[String] = Seq(

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index b26a9ab..b2ce3e0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -22,7 +22,6 @@ import java.net.{URL, URLClassLoader}
 import java.nio.charset.StandardCharsets
 import java.sql.Timestamp
 import java.util.concurrent.TimeUnit
-import java.util.regex.Pattern
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.HashMap
@@ -32,26 +31,18 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hive.common.`type`.HiveDecimal
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution
 import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable}
 import org.apache.hadoop.util.VersionInfo
 
 import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config.ConfigEntry
+import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression}
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.command.{ExecutedCommand, SetCommand}
 import org.apache.spark.sql.hive.client._
-import org.apache.spark.sql.hive.execution.{AnalyzeTable, DescribeHiveTableCommand, HiveNativeCommand}
-import org.apache.spark.sql.internal.{SharedState, SQLConf}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.internal.SQLConf._
 import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.UTF8String
 import org.apache.spark.util.Utils
 
 /**
@@ -61,225 +52,45 @@ import org.apache.spark.util.Utils
  * @since 1.0.0
  */
 class HiveContext private[hive](
-    @transient protected[hive] val hiveSharedState: HiveSharedState,
-    override val isRootContext: Boolean)
-  extends SQLContext(hiveSharedState, isRootContext) with Logging {
+    @transient private val sparkSession: SparkSession,
+    isRootContext: Boolean)
+  extends SQLContext(sparkSession, isRootContext) with Logging {
 
   self =>
 
   def this(sc: SparkContext) = {
-    this(new HiveSharedState(sc), true)
+    this(new SparkSession(HiveContext.withHiveExternalCatalog(sc)), true)
   }
 
   def this(sc: JavaSparkContext) = this(sc.sc)
 
-  import org.apache.spark.sql.hive.HiveContext._
-
-  logDebug("create HiveContext")
-
   /**
    * Returns a new HiveContext as new session, which will have separated SQLConf, UDF/UDAF,
    * temporary tables and SessionState, but sharing the same CacheManager, IsolatedClientLoader
    * and Hive client (both of execution and metadata) with existing HiveContext.
    */
   override def newSession(): HiveContext = {
-    new HiveContext(hiveSharedState, isRootContext = false)
-  }
-
-  @transient
-  protected[sql] override lazy val sessionState = new HiveSessionState(self)
-
-  protected[hive] def hiveCatalog: HiveExternalCatalog = hiveSharedState.externalCatalog
-  protected[hive] def executionHive: HiveClientImpl = sessionState.executionHive
-  protected[hive] def metadataHive: HiveClient = sessionState.metadataHive
-
-  /**
-   * When true, enables an experimental feature where metastore tables that use the parquet SerDe
-   * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive
-   * SerDe.
-   */
-  protected[sql] def convertMetastoreParquet: Boolean = getConf(CONVERT_METASTORE_PARQUET)
-
-  /**
-   * When true, also tries to merge possibly different but compatible Parquet schemas in different
-   * Parquet data files.
-   *
-   * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true.
-   */
-  protected[sql] def convertMetastoreParquetWithSchemaMerging: Boolean =
-    getConf(CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING)
-
-  /**
-   * When true, enables an experimental feature where metastore tables that use the Orc SerDe
-   * are automatically converted to use the Spark SQL ORC table scan, instead of the Hive
-   * SerDe.
-   */
-  protected[sql] def convertMetastoreOrc: Boolean = getConf(CONVERT_METASTORE_ORC)
-
-  /**
-   * When true, a table created by a Hive CTAS statement (no USING clause) will be
-   * converted to a data source table, using the data source set by spark.sql.sources.default.
-   * The table in CTAS statement will be converted when it meets any of the following conditions:
-   *   - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or
-   *     a Storage Hanlder (STORED BY), and the value of hive.default.fileformat in hive-site.xml
-   *     is either TextFile or SequenceFile.
-   *   - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe
-   *     is specified (no ROW FORMAT SERDE clause).
-   *   - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as the file format
-   *     and no SerDe is specified (no ROW FORMAT SERDE clause).
-   */
-  protected[sql] def convertCTAS: Boolean = getConf(CONVERT_CTAS)
-
-  /*
-   * hive thrift server use background spark sql thread pool to execute sql queries
-   */
-  protected[hive] def hiveThriftServerAsync: Boolean = getConf(HIVE_THRIFT_SERVER_ASYNC)
-
-  protected[hive] def hiveThriftServerSingleSession: Boolean =
-    sparkContext.conf.getBoolean("spark.sql.hive.thriftServer.singleSession", defaultValue = false)
-
-  @transient
-  protected[sql] lazy val substitutor = new VariableSubstitution()
-
-  /**
-   * Overrides default Hive configurations to avoid breaking changes to Spark SQL users.
-   *  - allow SQL11 keywords to be used as identifiers
-   */
-  private[sql] def defaultOverrides() = {
-    setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false")
-  }
-
-  defaultOverrides()
-
-  protected[sql] override def parseSql(sql: String): LogicalPlan = {
-    executionHive.withHiveState {
-      super.parseSql(substitutor.substitute(sessionState.hiveconf, sql))
-    }
-  }
-
-  override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution =
-    new this.QueryExecution(plan)
-
-  /**
-   * Invalidate and refresh all the cached the metadata of the given table. For performance reasons,
-   * Spark SQL or the external data source library it uses might cache certain metadata about a
-   * table, such as the location of blocks. When those change outside of Spark SQL, users should
-   * call this function to invalidate the cache.
-   *
-   * @since 1.3.0
-   */
-  def refreshTable(tableName: String): Unit = {
-    val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName)
-    sessionState.catalog.refreshTable(tableIdent)
-  }
-
-  protected[hive] def invalidateTable(tableName: String): Unit = {
-    val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName)
-    sessionState.catalog.invalidateTable(tableIdent)
-  }
-
-  /**
-   * Analyzes the given table in the current database to generate statistics, which will be
-   * used in query optimizations.
-   *
-   * Right now, it only supports Hive tables and it only updates the size of a Hive table
-   * in the Hive metastore.
-   *
-   * @since 1.2.0
-   */
-  def analyze(tableName: String) {
-    AnalyzeTable(tableName).run(self)
-  }
-
-  override def setConf(key: String, value: String): Unit = {
-    super.setConf(key, value)
-    executionHive.runSqlHive(s"SET $key=$value")
-    metadataHive.runSqlHive(s"SET $key=$value")
-    // If users put any Spark SQL setting in the spark conf (e.g. spark-defaults.conf),
-    // this setConf will be called in the constructor of the SQLContext.
-    // Also, calling hiveconf will create a default session containing a HiveConf, which
-    // will interfer with the creation of executionHive (which is a lazy val). So,
-    // we put hiveconf.set at the end of this method.
-    sessionState.hiveconf.set(key, value)
+    new HiveContext(sparkSession.newSession(), isRootContext = false)
   }
 
-  override private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = {
-    setConf(entry.key, entry.stringConverter(value))
+  protected[sql] override def sessionState: HiveSessionState = {
+    sparkSession.sessionState.asInstanceOf[HiveSessionState]
   }
 
-  private def functionOrMacroDDLPattern(command: String) = Pattern.compile(
-    ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL).matcher(command)
-
-  protected[hive] def runSqlHive(sql: String): Seq[String] = {
-    val command = sql.trim.toLowerCase
-    if (functionOrMacroDDLPattern(command).matches()) {
-      executionHive.runSqlHive(sql)
-    } else if (command.startsWith("set")) {
-      metadataHive.runSqlHive(sql)
-      executionHive.runSqlHive(sql)
-    } else {
-      metadataHive.runSqlHive(sql)
-    }
+  protected[sql] override def sharedState: HiveSharedState = {
+    sparkSession.sharedState.asInstanceOf[HiveSharedState]
   }
 
-  /**
-   * Executes a SQL query without parsing it, but instead passing it directly to Hive.
-   * This is currently only used for DDLs and will be removed as soon as Spark can parse
-   * all supported Hive DDLs itself.
-   */
-  protected[sql] override def runNativeSql(sqlText: String): Seq[Row] = {
-    runSqlHive(sqlText).map { s => Row(s) }
-  }
+}
 
-  /** Extends QueryExecution with hive specific features. */
-  protected[sql] class QueryExecution(logicalPlan: LogicalPlan)
-    extends org.apache.spark.sql.execution.QueryExecution(this, logicalPlan) {
-
-    /**
-     * Returns the result as a hive compatible sequence of strings.  For native commands, the
-     * execution is simply passed back to Hive.
-     */
-    def stringResult(): Seq[String] = executedPlan match {
-      case ExecutedCommand(desc: DescribeHiveTableCommand) =>
-        // If it is a describe command for a Hive table, we want to have the output format
-        // be similar with Hive.
-        desc.run(self).map {
-          case Row(name: String, dataType: String, comment) =>
-            Seq(name, dataType,
-              Option(comment.asInstanceOf[String]).getOrElse(""))
-              .map(s => String.format(s"%-20s", s))
-              .mkString("\t")
-        }
-      case command: ExecutedCommand =>
-        command.executeCollect().map(_.getString(0))
-
-      case other =>
-        val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq
-        // We need the types so we can output struct field names
-        val types = analyzed.output.map(_.dataType)
-        // Reformat to match hive tab delimited output.
-        result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq
-    }
 
-    override def simpleString: String =
-      logical match {
-        case _: HiveNativeCommand => "<Native command: executed by Hive>"
-        case _: SetCommand => "<SET command: executed by Hive, and noted by SQLContext>"
-        case _ => super.simpleString
-      }
-  }
+private[hive] object HiveContext extends Logging {
 
-  protected[sql] override def addJar(path: String): Unit = {
-    // Add jar to Hive and classloader
-    executionHive.addJar(path)
-    metadataHive.addJar(path)
-    Thread.currentThread().setContextClassLoader(executionHive.clientLoader.classLoader)
-    super.addJar(path)
+  def withHiveExternalCatalog(sc: SparkContext): SparkContext = {
+    sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive")
+    sc
   }
-}
-
 
-private[hive] object HiveContext extends Logging {
   /** The version of hive used internally by Spark SQL. */
   val hiveExecutionVersion: String = "1.2.1"
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 753950f..33a926e 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -115,17 +115,16 @@ private[hive] object HiveSerDe {
  * This is still used for things like creating data source tables, but in the future will be
  * cleaned up to integrate more nicely with [[HiveExternalCatalog]].
  */
-private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveContext)
-  extends Logging {
-
-  val conf = hive.conf
+private[hive] class HiveMetastoreCatalog(hive: SQLContext) extends Logging {
+  private val conf = hive.conf
+  private val sessionState = hive.sessionState.asInstanceOf[HiveSessionState]
+  private val client = hive.sharedState.asInstanceOf[HiveSharedState].metadataHive
+  private val hiveconf = sessionState.hiveconf
 
   /** A fully qualified identifier for a table (i.e., database.tableName) */
   case class QualifiedTableName(database: String, name: String)
 
-  private def getCurrentDatabase: String = {
-    hive.sessionState.catalog.getCurrentDatabase
-  }
+  private def getCurrentDatabase: String = hive.sessionState.catalog.getCurrentDatabase
 
   def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = {
     QualifiedTableName(
@@ -298,7 +297,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte
       CatalogTableType.MANAGED_TABLE
     }
 
-    val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.sessionState.hiveconf)
+    val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hiveconf)
     val dataSource =
       DataSource(
         hive,
@@ -600,14 +599,14 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte
   object ParquetConversions extends Rule[LogicalPlan] {
     private def shouldConvertMetastoreParquet(relation: MetastoreRelation): Boolean = {
       relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") &&
-        hive.convertMetastoreParquet
+        sessionState.convertMetastoreParquet
     }
 
     private def convertToParquetRelation(relation: MetastoreRelation): LogicalRelation = {
       val defaultSource = new ParquetDefaultSource()
       val fileFormatClass = classOf[ParquetDefaultSource]
 
-      val mergeSchema = hive.convertMetastoreParquetWithSchemaMerging
+      val mergeSchema = sessionState.convertMetastoreParquetWithSchemaMerging
       val options = Map(
         ParquetRelation.MERGE_SCHEMA -> mergeSchema.toString,
         ParquetRelation.METASTORE_TABLE_NAME -> TableIdentifier(
@@ -652,7 +651,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte
   object OrcConversions extends Rule[LogicalPlan] {
     private def shouldConvertMetastoreOrc(relation: MetastoreRelation): Boolean = {
       relation.tableDesc.getSerdeClassName.toLowerCase.contains("orc") &&
-        hive.convertMetastoreOrc
+        sessionState.convertMetastoreOrc
     }
 
     private def convertToOrcRelation(relation: MetastoreRelation): LogicalRelation = {
@@ -727,7 +726,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte
 
         val desc = table.copy(schema = schema)
 
-        if (hive.convertCTAS && table.storage.serde.isEmpty) {
+        if (sessionState.convertCTAS && table.storage.serde.isEmpty) {
           // Do the conversion when spark.sql.hive.convertCTAS is true and the query
           // does not specify any storage format (file format and storage handler).
           if (table.identifier.database.isDefined) {
@@ -815,14 +814,13 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte
  * the information from the metastore.
  */
 class MetaStoreFileCatalog(
-    hive: HiveContext,
+    ctx: SQLContext,
     paths: Seq[Path],
     partitionSpecFromHive: PartitionSpec)
-  extends HDFSFileCatalog(hive, Map.empty, paths, Some(partitionSpecFromHive.partitionColumns)) {
-
+  extends HDFSFileCatalog(ctx, Map.empty, paths, Some(partitionSpecFromHive.partitionColumns)) {
 
   override def getStatus(path: Path): Array[FileStatus] = {
-    val fs = path.getFileSystem(hive.sparkContext.hadoopConfiguration)
+    val fs = path.getFileSystem(ctx.sparkContext.hadoopConfiguration)
     fs.listStatus(path)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala
new file mode 100644
index 0000000..1c1bfb6
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala
@@ -0,0 +1,66 @@
+/*
+ * 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
+
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.execution.command.{ExecutedCommand, SetCommand}
+import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand}
+
+
+/**
+ * A [[QueryExecution]] with hive specific features.
+ */
+protected[hive] class HiveQueryExecution(ctx: SQLContext, logicalPlan: LogicalPlan)
+  extends QueryExecution(ctx, logicalPlan) {
+
+  /**
+   * Returns the result as a hive compatible sequence of strings.  For native commands, the
+   * execution is simply passed back to Hive.
+   */
+  def stringResult(): Seq[String] = executedPlan match {
+    case ExecutedCommand(desc: DescribeHiveTableCommand) =>
+      // If it is a describe command for a Hive table, we want to have the output format
+      // be similar with Hive.
+      desc.run(ctx).map {
+        case Row(name: String, dataType: String, comment) =>
+          Seq(name, dataType,
+            Option(comment.asInstanceOf[String]).getOrElse(""))
+            .map(s => String.format(s"%-20s", s))
+            .mkString("\t")
+      }
+    case command: ExecutedCommand =>
+      command.executeCollect().map(_.getString(0))
+
+    case other =>
+      val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq
+      // We need the types so we can output struct field names
+      val types = analyzed.output.map(_.dataType)
+      // Reformat to match hive tab delimited output.
+      result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq
+  }
+
+  override def simpleString: String =
+    logical match {
+      case _: HiveNativeCommand => "<Native command: executed by Hive>"
+      case _: SetCommand => "<SET command: executed by Hive, and noted by SQLContext>"
+      case _ => super.simpleString
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
index f91393f..4f95133 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.{UDAF, UDF}
 import org.apache.hadoop.hive.ql.exec.{FunctionRegistry => HiveFunctionRegistry}
 import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF}
 
+import org.apache.spark.sql.{AnalysisException, SQLContext}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
 import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
@@ -33,7 +34,6 @@ import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, SessionCat
 import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo}
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.execution.datasources.BucketSpec
 import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
 import org.apache.spark.sql.hive.client.HiveClient
@@ -45,10 +45,11 @@ import org.apache.spark.util.Utils
 private[sql] class HiveSessionCatalog(
     externalCatalog: HiveExternalCatalog,
     client: HiveClient,
-    context: HiveContext,
+    context: SQLContext,
     functionResourceLoader: FunctionResourceLoader,
     functionRegistry: FunctionRegistry,
-    conf: SQLConf)
+    conf: SQLConf,
+    hiveconf: HiveConf)
   extends SessionCatalog(externalCatalog, functionResourceLoader, functionRegistry, conf) {
 
   override def setCurrentDatabase(db: String): Unit = {
@@ -75,7 +76,7 @@ private[sql] class HiveSessionCatalog(
   // ----------------------------------------------------------------
 
   override def getDefaultDBPath(db: String): String = {
-    val defaultPath = context.sessionState.hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE)
+    val defaultPath = hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE)
     new Path(new Path(defaultPath), db + ".db").toString
   }
 
@@ -83,7 +84,7 @@ private[sql] class HiveSessionCatalog(
   // essentially a cache for metastore tables. However, it relies on a lot of session-specific
   // things so it would be a lot of work to split its functionality between HiveSessionCatalog
   // and HiveCatalog. We should still do it at some point...
-  private val metastoreCatalog = new HiveMetastoreCatalog(client, context)
+  private val metastoreCatalog = new HiveMetastoreCatalog(context)
 
   val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions
   val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
index 2b84852..09297c2 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
@@ -17,62 +17,80 @@
 
 package org.apache.spark.sql.hive
 
+import java.util.regex.Pattern
+
 import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+import org.apache.hadoop.hive.ql.parse.VariableSubstitution
 
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.analysis.Analyzer
 import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkPlanner
 import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.hive.client.{HiveClient, HiveClientImpl}
-import org.apache.spark.sql.hive.execution.HiveSqlParser
+import org.apache.spark.sql.hive.execution.{AnalyzeTable, HiveSqlParser}
 import org.apache.spark.sql.internal.{SessionState, SQLConf}
 
 
 /**
  * A class that holds all session-specific state in a given [[HiveContext]].
  */
-private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) {
+private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) {
 
-  /**
-   * SQLConf and HiveConf contracts:
-   *
-   * 1. create a new o.a.h.hive.ql.session.SessionState for each [[HiveContext]]
-   * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the
-   *    SQLConf.  Additionally, any properties set by set() or a SET command inside sql() will be
-   *    set in the SQLConf *as well as* in the HiveConf.
-   */
-  lazy val hiveconf: HiveConf = {
-    val c = ctx.executionHive.conf
-    ctx.setConf(c.getAllProperties)
-    c
-  }
+  self =>
+
+  private lazy val sharedState: HiveSharedState = ctx.sharedState.asInstanceOf[HiveSharedState]
 
   /**
    * A Hive client used for execution.
    */
-  val executionHive: HiveClientImpl = ctx.hiveSharedState.executionHive.newSession()
+  lazy val executionHive: HiveClientImpl = sharedState.executionHive.newSession()
 
   /**
    * A Hive client used for interacting with the metastore.
    */
-  val metadataHive: HiveClient = ctx.hiveSharedState.metadataHive.newSession()
+  lazy val metadataHive: HiveClient = sharedState.metadataHive.newSession()
+
+  /**
+   * A Hive helper class for substituting variables in a SQL statement.
+   */
+  lazy val substitutor = new VariableSubstitution
 
   override lazy val conf: SQLConf = new SQLConf {
     override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false)
   }
 
+
+  /**
+   * SQLConf and HiveConf contracts:
+   *
+   * 1. create a new o.a.h.hive.ql.session.SessionState for each HiveContext
+   * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the
+   *    SQLConf.  Additionally, any properties set by set() or a SET command inside sql() will be
+   *    set in the SQLConf *as well as* in the HiveConf.
+   */
+  lazy val hiveconf: HiveConf = {
+    val c = executionHive.conf
+    conf.setConf(c.getAllProperties)
+    c
+  }
+
+  setDefaultOverrideConfs()
+
   /**
    * Internal catalog for managing table and database states.
    */
   override lazy val catalog = {
     new HiveSessionCatalog(
-      ctx.hiveCatalog,
-      ctx.metadataHive,
+      sharedState.externalCatalog,
+      metadataHive,
       ctx,
       ctx.functionResourceLoader,
       functionRegistry,
-      conf)
+      conf,
+      hiveconf)
   }
 
   /**
@@ -96,7 +114,7 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx)
   /**
    * Parser for HiveQl query texts.
    */
-  override lazy val sqlParser: ParserInterface = new HiveSqlParser(hiveconf)
+  override lazy val sqlParser: ParserInterface = new HiveSqlParser(substitutor, hiveconf)
 
   /**
    * Planner that takes into account Hive-specific strategies.
@@ -104,13 +122,14 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx)
   override def planner: SparkPlanner = {
     new SparkPlanner(ctx.sparkContext, conf, experimentalMethods.extraStrategies)
       with HiveStrategies {
-      override val hiveContext = ctx
+      override val context: SQLContext = ctx
+      override val hiveconf: HiveConf = self.hiveconf
 
       override def strategies: Seq[Strategy] = {
         experimentalMethods.extraStrategies ++ Seq(
           FileSourceStrategy,
           DataSourceStrategy,
-          HiveCommandStrategy(ctx),
+          HiveCommandStrategy,
           HiveDDLStrategy,
           DDLStrategy,
           SpecialLimits,
@@ -130,4 +149,119 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx)
     }
   }
 
+
+  // ------------------------------------------------------
+  //  Helper methods, partially leftover from pre-2.0 days
+  // ------------------------------------------------------
+
+  override def executePlan(plan: LogicalPlan): HiveQueryExecution = {
+    new HiveQueryExecution(ctx, plan)
+  }
+
+  /**
+   * Overrides default Hive configurations to avoid breaking changes to Spark SQL users.
+   *  - allow SQL11 keywords to be used as identifiers
+   */
+  def setDefaultOverrideConfs(): Unit = {
+    setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false")
+  }
+
+  override def setConf(key: String, value: String): Unit = {
+    super.setConf(key, value)
+    executionHive.runSqlHive(s"SET $key=$value")
+    metadataHive.runSqlHive(s"SET $key=$value")
+    hiveconf.set(key, value)
+  }
+
+  override def addJar(path: String): Unit = {
+    super.addJar(path)
+    executionHive.addJar(path)
+    metadataHive.addJar(path)
+    Thread.currentThread().setContextClassLoader(executionHive.clientLoader.classLoader)
+  }
+
+  /**
+   * Analyzes the given table in the current database to generate statistics, which will be
+   * used in query optimizations.
+   *
+   * Right now, it only supports Hive tables and it only updates the size of a Hive table
+   * in the Hive metastore.
+   */
+  override def analyze(tableName: String): Unit = {
+    AnalyzeTable(tableName).run(ctx)
+  }
+
+  /**
+   * Execute a SQL statement by passing the query text directly to Hive.
+   */
+  override def runNativeSql(sql: String): Seq[String] = {
+    val command = sql.trim.toLowerCase
+    val functionOrMacroDDLPattern = Pattern.compile(
+      ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL)
+    if (functionOrMacroDDLPattern.matcher(command).matches()) {
+      executionHive.runSqlHive(sql)
+    } else if (command.startsWith("set")) {
+      metadataHive.runSqlHive(sql)
+      executionHive.runSqlHive(sql)
+    } else {
+      metadataHive.runSqlHive(sql)
+    }
+  }
+
+  /**
+   * When true, enables an experimental feature where metastore tables that use the parquet SerDe
+   * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive
+   * SerDe.
+   */
+  def convertMetastoreParquet: Boolean = {
+    conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET)
+  }
+
+  /**
+   * When true, also tries to merge possibly different but compatible Parquet schemas in different
+   * Parquet data files.
+   *
+   * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true.
+   */
+  def convertMetastoreParquetWithSchemaMerging: Boolean = {
+    conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING)
+  }
+
+  /**
+   * When true, enables an experimental feature where metastore tables that use the Orc SerDe
+   * are automatically converted to use the Spark SQL ORC table scan, instead of the Hive
+   * SerDe.
+   */
+  def convertMetastoreOrc: Boolean = {
+    conf.getConf(HiveContext.CONVERT_METASTORE_ORC)
+  }
+
+  /**
+   * When true, a table created by a Hive CTAS statement (no USING clause) will be
+   * converted to a data source table, using the data source set by spark.sql.sources.default.
+   * The table in CTAS statement will be converted when it meets any of the following conditions:
+   *   - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or
+   *     a Storage Hanlder (STORED BY), and the value of hive.default.fileformat in hive-site.xml
+   *     is either TextFile or SequenceFile.
+   *   - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe
+   *     is specified (no ROW FORMAT SERDE clause).
+   *   - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as the file format
+   *     and no SerDe is specified (no ROW FORMAT SERDE clause).
+   */
+  def convertCTAS: Boolean = {
+    conf.getConf(HiveContext.CONVERT_CTAS)
+  }
+
+  /**
+   * When true, Hive Thrift server will execute SQL queries asynchronously using a thread pool."
+   */
+  def hiveThriftServerAsync: Boolean = {
+    conf.getConf(HiveContext.HIVE_THRIFT_SERVER_ASYNC)
+  }
+
+  def hiveThriftServerSingleSession: Boolean = {
+    ctx.sparkContext.conf.getBoolean(
+      "spark.sql.hive.thriftServer.singleSession", defaultValue = false)
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index 010361a..bbdcc8c 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.hive
 
+import org.apache.hadoop.hive.conf.HiveConf
+
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.planning._
@@ -31,12 +33,13 @@ private[hive] trait HiveStrategies {
   // Possibly being too clever with types here... or not clever enough.
   self: SparkPlanner =>
 
-  val hiveContext: HiveContext
+  val context: SQLContext
+  val hiveconf: HiveConf
 
   object Scripts extends Strategy {
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
       case logical.ScriptTransformation(input, script, output, child, schema: HiveScriptIOSchema) =>
-        ScriptTransformation(input, script, output, planLater(child), schema)(hiveContext) :: Nil
+        ScriptTransformation(input, script, output, planLater(child), schema)(hiveconf) :: Nil
       case _ => Nil
     }
   }
@@ -74,7 +77,7 @@ private[hive] trait HiveStrategies {
           projectList,
           otherPredicates,
           identity[Seq[Expression]],
-          HiveTableScan(_, relation, pruningPredicates)(hiveContext)) :: Nil
+          HiveTableScan(_, relation, pruningPredicates)(context, hiveconf)) :: Nil
       case _ =>
         Nil
     }
@@ -103,7 +106,7 @@ private[hive] trait HiveStrategies {
     }
   }
 
-  case class HiveCommandStrategy(context: HiveContext) extends Strategy {
+  case object HiveCommandStrategy extends Strategy {
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
       case describe: DescribeCommand =>
         ExecutedCommand(

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index 98a4273..6a20d7c 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -37,6 +37,7 @@ import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.internal.Logging
 import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD}
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
@@ -61,8 +62,8 @@ private[hive]
 class HadoopTableReader(
     @transient private val attributes: Seq[Attribute],
     @transient private val relation: MetastoreRelation,
-    @transient private val sc: HiveContext,
-    hiveExtraConf: HiveConf)
+    @transient private val sc: SQLContext,
+    hiveconf: HiveConf)
   extends TableReader with Logging {
 
   // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local".
@@ -72,14 +73,12 @@ class HadoopTableReader(
   private val _minSplitsPerRDD = if (sc.sparkContext.isLocal) {
     0 // will splitted based on block by default.
   } else {
-    math.max(
-      sc.sessionState.hiveconf.getInt("mapred.map.tasks", 1),
-      sc.sparkContext.defaultMinPartitions)
+    math.max(hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions)
   }
 
-  SparkHadoopUtil.get.appendS3AndSparkHadoopConfigurations(sc.sparkContext.conf, hiveExtraConf)
+  SparkHadoopUtil.get.appendS3AndSparkHadoopConfigurations(sc.sparkContext.conf, hiveconf)
   private val _broadcastedHiveConf =
-    sc.sparkContext.broadcast(new SerializableConfiguration(hiveExtraConf))
+    sc.sparkContext.broadcast(new SerializableConfiguration(hiveconf))
 
   override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] =
     makeRDDForTable(
@@ -164,7 +163,7 @@ class HadoopTableReader(
           case (partition, partDeserializer) =>
             def updateExistPathSetByPathPattern(pathPatternStr: String) {
               val pathPattern = new Path(pathPatternStr)
-              val fs = pathPattern.getFileSystem(sc.sessionState.hiveconf)
+              val fs = pathPattern.getFileSystem(hiveconf)
               val matches = fs.globStatus(pathPattern)
               matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString)
             }
@@ -261,7 +260,7 @@ class HadoopTableReader(
   private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = {
     filterOpt match {
       case Some(filter) =>
-        val fs = path.getFileSystem(sc.sessionState.hiveconf)
+        val fs = path.getFileSystem(hiveconf)
         val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString)
         filteredFiles.mkString(",")
       case None => path.toString

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
index 2a1fff9..69f7dbf 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
@@ -151,6 +151,8 @@ private[hive] class HiveClientImpl(
   }
 
   /** Returns the configuration for the current session. */
+  // TODO: We should not use it because HiveSessionState has a hiveconf
+  // for the current Session.
   def conf: HiveConf = SessionState.get().getConf
 
   override def getConf(key: String, defaultValue: String): String = {

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index 29f7dc2..ceb7f3b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -43,7 +43,6 @@ case class CreateTableAsSelect(
   override def children: Seq[LogicalPlan] = Seq(query)
 
   override def run(sqlContext: SQLContext): Seq[Row] = {
-    val hiveContext = sqlContext.asInstanceOf[HiveContext]
     lazy val metastoreRelation: MetastoreRelation = {
       import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
       import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
@@ -69,24 +68,24 @@ case class CreateTableAsSelect(
         withFormat
       }
 
-      hiveContext.sessionState.catalog.createTable(withSchema, ignoreIfExists = false)
+      sqlContext.sessionState.catalog.createTable(withSchema, ignoreIfExists = false)
 
       // Get the Metastore Relation
-      hiveContext.sessionState.catalog.lookupRelation(tableIdentifier) match {
+      sqlContext.sessionState.catalog.lookupRelation(tableIdentifier) match {
         case r: MetastoreRelation => r
       }
     }
     // TODO ideally, we should get the output data ready first and then
     // add the relation into catalog, just in case of failure occurs while data
     // processing.
-    if (hiveContext.sessionState.catalog.tableExists(tableIdentifier)) {
+    if (sqlContext.sessionState.catalog.tableExists(tableIdentifier)) {
       if (allowExisting) {
         // table already exists, will do nothing, to keep consistent with Hive
       } else {
         throw new AnalysisException(s"$tableIdentifier already exists.")
       }
     } else {
-      hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd
+      sqlContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd
     }
 
     Seq.empty[Row]

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala
index 33cd8b4..1e234d8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala
@@ -20,12 +20,11 @@ package org.apache.spark.sql.hive.execution
 import scala.util.control.NonFatal
 
 import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
-import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable}
 import org.apache.spark.sql.catalyst.expressions.Alias
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
 import org.apache.spark.sql.execution.command.RunnableCommand
-import org.apache.spark.sql.hive.{ HiveContext, HiveMetastoreTypes, SQLBuilder}
+import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveSessionState, SQLBuilder}
 
 /**
  * Create Hive view on non-hive-compatible tables by specifying schema ourselves instead of
@@ -47,16 +46,16 @@ private[hive] case class CreateViewAsSelect(
   private val tableIdentifier = tableDesc.identifier
 
   override def run(sqlContext: SQLContext): Seq[Row] = {
-    val hiveContext = sqlContext.asInstanceOf[HiveContext]
+    val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState]
 
-    hiveContext.sessionState.catalog.tableExists(tableIdentifier) match {
+    sessionState.catalog.tableExists(tableIdentifier) match {
       case true if allowExisting =>
         // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view
         // already exists.
 
       case true if orReplace =>
         // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...`
-        hiveContext.metadataHive.alertView(prepareTable(sqlContext))
+        sessionState.metadataHive.alertView(prepareTable(sqlContext))
 
       case true =>
         // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already
@@ -66,7 +65,7 @@ private[hive] case class CreateViewAsSelect(
           "CREATE OR REPLACE VIEW AS")
 
       case false =>
-        hiveContext.metadataHive.createView(prepareTable(sqlContext))
+        sessionState.metadataHive.createView(prepareTable(sqlContext))
     }
 
     Seq.empty[Row]

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
index 9bb9719..8c1f4a8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.execution
 import org.apache.spark.sql.{Row, SQLContext}
 import org.apache.spark.sql.catalyst.expressions.AttributeReference
 import org.apache.spark.sql.execution.command.RunnableCommand
-import org.apache.spark.sql.hive.HiveContext
+import org.apache.spark.sql.hive.HiveSessionState
 import org.apache.spark.sql.types.StringType
 
 private[hive]
@@ -29,6 +29,8 @@ case class HiveNativeCommand(sql: String) extends RunnableCommand {
   override def output: Seq[AttributeReference] =
     Seq(AttributeReference("result", StringType, nullable = false)())
 
-  override def run(sqlContext: SQLContext): Seq[Row] =
-    sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_))
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+    sqlContext.sessionState.asInstanceOf[HiveSessionState].runNativeSql(sql).map(Row(_))
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
index d5d3ee4..4ff02cd 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
@@ -21,8 +21,7 @@ import scala.collection.JavaConverters._
 import org.antlr.v4.runtime.{ParserRuleContext, Token}
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars
-import org.apache.hadoop.hive.ql.parse.EximUtil
-import org.apache.hadoop.hive.ql.session.SessionState
+import org.apache.hadoop.hive.ql.parse.{EximUtil, VariableSubstitution}
 import org.apache.hadoop.hive.serde.serdeConstants
 import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
 
@@ -39,11 +38,19 @@ import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
 /**
  * Concrete parser for HiveQl statements.
  */
-class HiveSqlParser(hiveConf: HiveConf) extends AbstractSqlParser {
-  val astBuilder = new HiveSqlAstBuilder(hiveConf)
+class HiveSqlParser(
+    substitutor: VariableSubstitution,
+    hiveconf: HiveConf)
+  extends AbstractSqlParser {
 
-  override protected def nativeCommand(sqlText: String): LogicalPlan = {
-    HiveNativeCommand(sqlText)
+  val astBuilder = new HiveSqlAstBuilder(hiveconf)
+
+  protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    super.parse(substitutor.substitute(hiveconf, command))(toResult)
+  }
+
+  protected override def nativeCommand(sqlText: String): LogicalPlan = {
+    HiveNativeCommand(substitutor.substitute(hiveconf, sqlText))
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
index 3c46b83..9a83466 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.Object
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
 
 import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution._
@@ -47,7 +48,8 @@ case class HiveTableScan(
     requestedAttributes: Seq[Attribute],
     relation: MetastoreRelation,
     partitionPruningPred: Seq[Expression])(
-    @transient val context: HiveContext)
+    @transient val context: SQLContext,
+    @transient val hiveconf: HiveConf)
   extends LeafNode {
 
   require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned,
@@ -75,7 +77,7 @@ case class HiveTableScan(
   // Create a local copy of hiveconf,so that scan specific modifications should not impact
   // other queries
   @transient
-  private[this] val hiveExtraConf = new HiveConf(context.sessionState.hiveconf)
+  private[this] val hiveExtraConf = new HiveConf(hiveconf)
 
   // append columns ids and names before broadcast
   addColumnMetadataToConf(hiveExtraConf)

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index ed53863..e614daa 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -43,9 +43,10 @@ case class InsertIntoHiveTable(
     overwrite: Boolean,
     ifNotExists: Boolean) extends UnaryNode {
 
-  @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext]
-  @transient private lazy val hiveContext = new Context(sc.sessionState.hiveconf)
-  @transient private lazy val client = sc.metadataHive
+  @transient private val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState]
+  @transient private val client = sessionState.metadataHive
+  @transient private val hiveconf = sessionState.hiveconf
+  @transient private lazy val hiveContext = new Context(hiveconf)
 
   def output: Seq[Attribute] = Seq.empty
 
@@ -67,7 +68,7 @@ case class InsertIntoHiveTable(
       SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName, conf.value))
     log.debug("Saving as hadoop file of type " + valueClass.getSimpleName)
     writerContainer.driverSideSetup()
-    sc.sparkContext.runJob(rdd, writerContainer.writeToFile _)
+    sqlContext.sparkContext.runJob(rdd, writerContainer.writeToFile _)
     writerContainer.commitJob()
 
   }
@@ -86,17 +87,17 @@ case class InsertIntoHiveTable(
     val tableLocation = table.hiveQlTable.getDataLocation
     val tmpLocation = hiveContext.getExternalTmpPath(tableLocation)
     val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false)
-    val isCompressed = sc.sessionState.hiveconf.getBoolean(
+    val isCompressed = hiveconf.getBoolean(
       ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal)
 
     if (isCompressed) {
       // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec",
       // and "mapred.output.compression.type" have no impact on ORC because it uses table properties
       // to store compression information.
-      sc.sessionState.hiveconf.set("mapred.output.compress", "true")
+      hiveconf.set("mapred.output.compress", "true")
       fileSinkConf.setCompressed(true)
-      fileSinkConf.setCompressCodec(sc.sessionState.hiveconf.get("mapred.output.compression.codec"))
-      fileSinkConf.setCompressType(sc.sessionState.hiveconf.get("mapred.output.compression.type"))
+      fileSinkConf.setCompressCodec(hiveconf.get("mapred.output.compression.codec"))
+      fileSinkConf.setCompressType(hiveconf.get("mapred.output.compression.type"))
     }
 
     val numDynamicPartitions = partition.values.count(_.isEmpty)
@@ -113,12 +114,12 @@ case class InsertIntoHiveTable(
     // Validate partition spec if there exist any dynamic partitions
     if (numDynamicPartitions > 0) {
       // Report error if dynamic partitioning is not enabled
-      if (!sc.sessionState.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) {
+      if (!hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) {
         throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg)
       }
 
       // Report error if dynamic partition strict mode is on but no static partition is found
-      if (numStaticPartitions == 0 && sc.sessionState.hiveconf.getVar(
+      if (numStaticPartitions == 0 && hiveconf.getVar(
           HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) {
         throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg)
       }
@@ -130,7 +131,7 @@ case class InsertIntoHiveTable(
       }
     }
 
-    val jobConf = new JobConf(sc.sessionState.hiveconf)
+    val jobConf = new JobConf(hiveconf)
     val jobConfSer = new SerializableJobConf(jobConf)
 
     // When speculation is on and output committer class name contains "Direct", we should warn

http://git-wip-us.apache.org/repos/asf/spark/blob/8fc267ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
index ea48b0e..2f7cec3 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
@@ -26,6 +26,7 @@ import scala.collection.JavaConverters._
 import scala.util.control.NonFatal
 
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter}
 import org.apache.hadoop.hive.serde.serdeConstants
 import org.apache.hadoop.hive.serde2.AbstractSerDe
@@ -39,7 +40,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema
 import org.apache.spark.sql.execution._
-import org.apache.spark.sql.hive.{HiveContext, HiveInspectors}
+import org.apache.spark.sql.hive.HiveInspectors
 import org.apache.spark.sql.hive.HiveShim._
 import org.apache.spark.sql.types.DataType
 import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfiguration, Utils}
@@ -57,14 +58,14 @@ case class ScriptTransformation(
     script: String,
     output: Seq[Attribute],
     child: SparkPlan,
-    ioschema: HiveScriptIOSchema)(@transient private val sc: HiveContext)
+    ioschema: HiveScriptIOSchema)(@transient private val hiveconf: HiveConf)
   extends UnaryNode {
 
-  override protected def otherCopyArgs: Seq[HiveContext] = sc :: Nil
+  override protected def otherCopyArgs: Seq[HiveConf] = hiveconf :: Nil
 
   override def producedAttributes: AttributeSet = outputSet -- inputSet
 
-  private val serializedHiveConf = new SerializableConfiguration(sc.sessionState.hiveconf)
+  private val serializedHiveConf = new SerializableConfiguration(hiveconf)
 
   protected override def doExecute(): RDD[InternalRow] = {
     def processIterator(inputIterator: Iterator[InternalRow]): Iterator[InternalRow] = {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org