From f6585f98962403247289c4352954579d14b4e2ed Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 10:45:43 -0700 Subject: [PATCH 01/27] [SPARK-14647][SQL] Group SQLContext/HiveContext state into SharedState --- .../org/apache/spark/sql/SQLContext.scala | 31 +++---- .../spark/sql/internal/SessionState.scala | 2 - .../spark/sql/internal/SharedState.scala | 47 ++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 51 ++++------- .../spark/sql/hive/HiveSessionState.scala | 15 +++- .../spark/sql/hive/HiveSharedState.scala | 53 ++++++++++++ .../apache/spark/sql/hive/test/TestHive.scala | 86 +++++++------------ .../sql/hive/HiveExternalCatalogSuite.scala | 12 +-- 8 files changed, 175 insertions(+), 122 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.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 9259ff40625c9..781d6998190b2 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 @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} -import org.apache.spark.sql.internal.{SessionState, SQLConf} +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ExecutionListenerManager @@ -63,17 +63,14 @@ import org.apache.spark.util.Utils * @since 1.0.0 */ class SQLContext private[sql]( - @transient val sparkContext: SparkContext, - @transient protected[sql] val cacheManager: CacheManager, - @transient private[sql] val listener: SQLListener, - val isRootContext: Boolean, - @transient private[sql] val externalCatalog: ExternalCatalog) + @transient protected[sql] val sharedState: SharedState, + val isRootContext: Boolean) extends Logging with Serializable { self => def this(sc: SparkContext) = { - this(sc, new CacheManager, SQLContext.createListenerAndUI(sc), true, new InMemoryCatalog) + this(new SharedState(sc), true) } def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) @@ -100,20 +97,20 @@ class SQLContext private[sql]( } } + def sparkContext: SparkContext = sharedState.sparkContext + + protected[sql] def cacheManager: CacheManager = sharedState.cacheManager + protected[sql] def listener: SQLListener = sharedState.listener + protected[sql] def externalCatalog: ExternalCatalog = sharedState.externalCatalog + /** - * Returns a SQLContext as new session, with separated SQL configurations, temporary tables, - * registered functions, but sharing the same SparkContext, CacheManager, SQLListener and SQLTab. + * Returns a [[SQLContext]] as new session, with separated SQL configurations, temporary + * tables, registered functions, but sharing the same [[SparkContext]], cached data and + * other things. * * @since 1.6.0 */ - def newSession(): SQLContext = { - new SQLContext( - sparkContext = sparkContext, - cacheManager = cacheManager, - listener = listener, - isRootContext = false, - externalCatalog = externalCatalog) - } + def newSession(): SQLContext = new SQLContext(sharedState, isRootContext = false) /** * Per-session state, e.g. configuration, functions, temporary tables etc. 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 c30f879dedaa1..d404a7c0aef59 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 @@ -22,10 +22,8 @@ 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.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, PreInsertCastAndRename, ResolveDataSource} -import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.util.ExecutionListenerManager /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala new file mode 100644 index 0000000000000..9a30c7de1f8f2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -0,0 +1,47 @@ +/* + * 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.internal + +import org.apache.spark.SparkContext +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} +import org.apache.spark.sql.execution.CacheManager +import org.apache.spark.sql.execution.ui.SQLListener + + +/** + * A class that holds all state shared across sessions in a given [[SQLContext]]. + */ +private[sql] class SharedState(val sparkContext: SparkContext) { + + /** + * Class for caching query results reused in future executions. + */ + val cacheManager: CacheManager = new CacheManager + + /** + * A listener for SQL-specific [[org.apache.spark.scheduler.SparkListenerEvent]]s. + */ + val listener: SQLListener = SQLContext.createListenerAndUI(sparkContext) + + /** + * A catalog that interacts with external systems. + */ + lazy val externalCatalog: ExternalCatalog = new InMemoryCatalog + +} 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 42cda0be161d3..71ef99a6a9952 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 @@ -45,12 +45,10 @@ 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._ import org.apache.spark.sql.execution.command.{ExecutedCommand, SetCommand} -import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{AnalyzeTable, DescribeHiveTableCommand, HiveNativeCommand} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SharedState, SQLConf} import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -63,32 +61,14 @@ import org.apache.spark.util.Utils * @since 1.0.0 */ class HiveContext private[hive]( - sc: SparkContext, - cacheManager: CacheManager, - listener: SQLListener, - @transient private[hive] val executionHive: HiveClientImpl, - @transient private[hive] val metadataHive: HiveClient, - isRootContext: Boolean, - @transient private[sql] val hiveCatalog: HiveExternalCatalog) - extends SQLContext(sc, cacheManager, listener, isRootContext, hiveCatalog) with Logging { - self => + @transient protected[hive] val hiveSharedState: HiveSharedState, + override val isRootContext: Boolean) + extends SQLContext(hiveSharedState, isRootContext) with Logging { - private def this(sc: SparkContext, execHive: HiveClientImpl, metaHive: HiveClient) { - this( - sc, - new CacheManager, - SQLContext.createListenerAndUI(sc), - execHive, - metaHive, - true, - new HiveExternalCatalog(metaHive)) - } + self => def this(sc: SparkContext) = { - this( - sc, - HiveContext.newClientForExecution(sc.conf, sc.hadoopConfiguration), - HiveContext.newClientForMetadata(sc.conf, sc.hadoopConfiguration)) + this(new HiveSharedState(sc), true) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -103,19 +83,16 @@ class HiveContext private[hive]( * and Hive client (both of execution and metadata) with existing HiveContext. */ override def newSession(): HiveContext = { - new HiveContext( - sc = sc, - cacheManager = cacheManager, - listener = listener, - executionHive = executionHive.newSession(), - metadataHive = metadataHive.newSession(), - isRootContext = false, - hiveCatalog = hiveCatalog) + 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 @@ -159,7 +136,7 @@ class HiveContext private[hive]( protected[hive] def hiveThriftServerAsync: Boolean = getConf(HIVE_THRIFT_SERVER_ASYNC) protected[hive] def hiveThriftServerSingleSession: Boolean = - sc.conf.get("spark.sql.hive.thriftServer.singleSession", "false").toBoolean + sparkContext.conf.getBoolean("spark.sql.hive.thriftServer.singleSession", defaultValue = false) @transient protected[sql] lazy val substitutor = new VariableSubstitution() @@ -527,7 +504,9 @@ private[hive] object HiveContext extends Logging { * The version of the Hive client that is used here must match the metastore that is configured * in the hive-site.xml file. */ - private def newClientForMetadata(conf: SparkConf, hadoopConf: Configuration): HiveClient = { + protected[hive] def newClientForMetadata( + conf: SparkConf, + hadoopConf: Configuration): HiveClient = { val hiveConf = new HiveConf(hadoopConf, classOf[HiveConf]) val configurations = hiveClientConfigurations(hiveConf) newClientForMetadata(conf, hiveConf, hadoopConf, configurations) 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 b992fda18cef7..bc28b55d06d9e 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 @@ -18,10 +18,11 @@ package org.apache.spark.sql.hive import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.execution.{python, SparkPlanner} +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.internal.{SessionState, SQLConf} @@ -31,6 +32,16 @@ import org.apache.spark.sql.internal.{SessionState, SQLConf} */ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) { + /** + * A Hive client used for execution. + */ + val executionHive: HiveClientImpl = ctx.hiveSharedState.executionHive.newSession() + + /** + * A Hive client used for interacting with the metastore. + */ + val metadataHive: HiveClient = ctx.hiveSharedState.metadataHive.newSession() + override lazy val conf: SQLConf = new SQLConf { override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala new file mode 100644 index 0000000000000..11097c33df2d5 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala @@ -0,0 +1,53 @@ +/* + * 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.SparkContext +import org.apache.spark.sql.hive.client.{HiveClient, HiveClientImpl} +import org.apache.spark.sql.internal.SharedState + + +/** + * A class that holds all state shared across sessions in a given [[HiveContext]]. + */ +private[hive] class HiveSharedState(override val sparkContext: SparkContext) + extends SharedState(sparkContext) { + + // TODO: just share the IsolatedClientLoader instead of the client instances themselves + + /** + * A Hive client used for execution. + */ + val executionHive: HiveClientImpl = { + HiveContext.newClientForExecution(sparkContext.conf, sparkContext.hadoopConfiguration) + } + + /** + * A Hive client used to interact with the metastore. + */ + // This needs to be a lazy val at here because TestHiveSharedState is overriding it. + lazy val metadataHive: HiveClient = { + HiveContext.newClientForMetadata(sparkContext.conf, sparkContext.hadoopConfiguration) + } + + /** + * A catalog that interacts with the Hive metastore. + */ + override lazy val externalCatalog = new HiveExternalCatalog(metadataHive) + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 7f6ca21782da4..d56d36fe32e77 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -72,63 +72,24 @@ object TestHive * test cases that rely on TestHive must be serialized. */ class TestHiveContext private[hive]( - sc: SparkContext, - cacheManager: CacheManager, - listener: SQLListener, - executionHive: HiveClientImpl, - metadataHive: HiveClient, - isRootContext: Boolean, - hiveCatalog: HiveExternalCatalog, + testHiveSharedState: TestHiveSharedState, val warehousePath: File, val scratchDirPath: File, - metastoreTemporaryConf: Map[String, String]) - extends HiveContext( - sc, - cacheManager, - listener, - executionHive, - metadataHive, - isRootContext, - hiveCatalog) { self => - - // Unfortunately, due to the complex interactions between the construction parameters - // and the limitations in scala constructors, we need many of these constructors to - // provide a shorthand to create a new TestHiveContext with only a SparkContext. - // This is not a great design pattern but it's necessary here. + metastoreTemporaryConf: Map[String, String], + isRootContext: Boolean) + extends HiveContext(testHiveSharedState, isRootContext) { self => private def this( sc: SparkContext, - executionHive: HiveClientImpl, - metadataHive: HiveClient, warehousePath: File, scratchDirPath: File, metastoreTemporaryConf: Map[String, String]) { this( - sc, - new CacheManager, - SQLContext.createListenerAndUI(sc), - executionHive, - metadataHive, - true, - new HiveExternalCatalog(metadataHive), + new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf), warehousePath, scratchDirPath, - metastoreTemporaryConf) - } - - private def this( - sc: SparkContext, - warehousePath: File, - scratchDirPath: File, - metastoreTemporaryConf: Map[String, String]) { - this( - sc, - HiveContext.newClientForExecution(sc.conf, sc.hadoopConfiguration), - TestHiveContext.newClientForMetadata( - sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath, metastoreTemporaryConf), - warehousePath, - scratchDirPath, - metastoreTemporaryConf) + metastoreTemporaryConf, + true) } def this(sc: SparkContext) { @@ -141,16 +102,11 @@ class TestHiveContext private[hive]( override def newSession(): HiveContext = { new TestHiveContext( - sc = sc, - cacheManager = cacheManager, - listener = listener, - executionHive = executionHive.newSession(), - metadataHive = metadataHive.newSession(), - isRootContext = false, - hiveCatalog = hiveCatalog, - warehousePath = warehousePath, - scratchDirPath = scratchDirPath, - metastoreTemporaryConf = metastoreTemporaryConf) + testHiveSharedState, + warehousePath, + scratchDirPath, + metastoreTemporaryConf, + isRootContext = false) } // By clearing the port we force Spark to pick a new one. This allows us to rerun tests @@ -549,6 +505,22 @@ private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { } } + +private[hive] class TestHiveSharedState( + sc: SparkContext, + warehousePath: File, + scratchDirPath: File, + metastoreTemporaryConf: Map[String, String]) + extends HiveSharedState(sc) { + + override lazy val metadataHive: HiveClient = { + TestHiveContext.newClientForMetadata( + sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath, metastoreTemporaryConf) + } + +} + + private[hive] object TestHiveContext { /** @@ -563,7 +535,7 @@ private[hive] object TestHiveContext { /** * Create a [[HiveClient]] used to retrieve metadata from the Hive MetaStore. */ - private def newClientForMetadata( + def newClientForMetadata( conf: SparkConf, hadoopConf: Configuration, warehousePath: File, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 3334c16f0be87..84285b7f40832 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -18,12 +18,10 @@ package org.apache.spark.sql.hive import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.util.VersionInfo import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.hive.client.{HiveClient, IsolatedClientLoader} -import org.apache.spark.util.Utils +import org.apache.spark.sql.hive.client.HiveClient /** * Test suite for the [[HiveExternalCatalog]]. @@ -31,11 +29,9 @@ import org.apache.spark.util.Utils class HiveExternalCatalogSuite extends CatalogTestCases { private val client: HiveClient = { - IsolatedClientLoader.forVersion( - hiveMetastoreVersion = HiveContext.hiveExecutionVersion, - hadoopVersion = VersionInfo.getVersion, - sparkConf = new SparkConf(), - hadoopConf = new Configuration()).createClient() + // We create a metastore at a temp location to avoid any potential + // conflict of having multiple connections to a single derby instance. + HiveContext.newClientForExecution(new SparkConf, new Configuration) } protected override val utils: CatalogTestUtils = new CatalogTestUtils { From fe89b8ddc44a32949be6b763e180ef6587d1e310 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 12:45:33 -0700 Subject: [PATCH 02/27] Make HiveSessionState take in SQLContext, not HiveContext This requires changing all the downstream places that take in HiveContext and replacing that with (SQLContext, HiveSessionState). --- .../org/apache/spark/sql/SQLContext.scala | 8 +- .../thriftserver/SparkSQLSessionManager.scala | 2 +- .../server/SparkSQLOperationManager.scala | 2 +- .../execution/HiveCompatibilitySuite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 81 +------------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 33 +++--- .../spark/sql/hive/HiveSessionCatalog.scala | 9 +- .../spark/sql/hive/HiveSessionState.scala | 100 ++++++++++++++++-- .../spark/sql/hive/HiveStrategies.scala | 11 +- .../apache/spark/sql/hive/TableReader.scala | 15 +-- .../sql/hive/execution/HiveSqlParser.scala | 37 +++---- .../sql/hive/execution/HiveTableScan.scala | 6 +- .../hive/execution/ScriptTransformation.scala | 9 +- .../apache/spark/sql/hive/test/TestHive.scala | 6 +- .../spark/sql/hive/ErrorPositionSuite.scala | 3 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 5 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../hive/execution/ConcurrentHiveSuite.scala | 6 +- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- .../execution/ScriptTransformationSuite.scala | 8 +- 20 files changed, 179 insertions(+), 170 deletions(-) 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 781d6998190b2..c51b1a71ff387 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 @@ -192,10 +192,6 @@ class SQLContext private[sql]( 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) /** @@ -759,7 +755,9 @@ class SQLContext private[sql]( * @since 1.3.0 */ def sql(sqlText: String): DataFrame = { - Dataset.ofRows(this, parseSql(sqlText)) + // TODO(andrew): delegate all parsing to the session!! + // Right now we're not substituting things correctly for HiveContext. + Dataset.ofRows(this, sessionState.sqlParser.parsePlan(sqlText)) } /** 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 de4e9c62b57a4..f492b5656c3c3 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() 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 0c468a408ba98..da410c68c851d 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) 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 989e68aebed9b..11c3d722b00a4 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) 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 71ef99a6a9952..2545bfcd7858c 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 @@ -32,7 +32,6 @@ 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 @@ -41,17 +40,13 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry 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 /** @@ -73,8 +68,6 @@ class HiveContext private[hive]( def this(sc: JavaSparkContext) = this(sc.sc) - import org.apache.spark.sql.hive.HiveContext._ - logDebug("create HiveContext") /** @@ -87,59 +80,12 @@ class HiveContext private[hive]( } @transient - protected[sql] override lazy val sessionState = new HiveSessionState(self) + protected[sql] override lazy val sessionState = new HiveSessionState(self, hiveSharedState) 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() + protected[hive] def hiveconf: HiveConf = sessionState.hiveconf /** * Overrides default Hive configurations to avoid breaking changes to Spark SQL users. @@ -151,12 +97,6 @@ class HiveContext private[hive]( defaultOverrides() - protected[sql] override def parseSql(sql: String): LogicalPlan = { - executionHive.withHiveState { - super.parseSql(substitutor.substitute(hiveconf, sql)) - } - } - override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) @@ -207,21 +147,6 @@ class HiveContext private[hive]( setConf(entry.key, entry.stringConverter(value)) } - /** - * 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. - */ - @transient - protected[hive] lazy val hiveconf: HiveConf = { - val c = executionHive.conf - setConf(c.getAllProperties) - c - } - private def functionOrMacroDDLPattern(command: String) = Pattern.compile( ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL).matcher(command) 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 ccc8345d7375d..9bf1c7bc9daff 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 @@ -116,7 +116,11 @@ 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) +// TODO(andrew): once we have SparkSession just pass that in here instead of the context +private[hive] class HiveMetastoreCatalog( + val client: HiveClient, + hive: SQLContext, + sessionState: HiveSessionState) extends Logging { val conf = hive.conf @@ -124,9 +128,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte /** 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 = sessionState.catalog.getCurrentDatabase def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = { QualifiedTableName( @@ -299,7 +301,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte CatalogTableType.MANAGED_TABLE } - val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.hiveconf) + val maybeSerDe = HiveSerDe.sourceToSerDe(provider, sessionState.hiveconf) val dataSource = DataSource( hive, @@ -443,8 +445,10 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." - case None => SubqueryAlias(table.identifier.table, hive.parseSql(viewText)) - case Some(aliasText) => SubqueryAlias(aliasText, hive.parseSql(viewText)) + case None => + SubqueryAlias(table.identifier.table, sessionState.sqlParser.parsePlan(viewText)) + case Some(aliasText) => + SubqueryAlias(aliasText, sessionState.sqlParser.parsePlan(viewText)) } } else { MetastoreRelation( @@ -600,14 +604,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 +656,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 +731,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 +819,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) } 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 0cccc22e5a624..9f3f4ce41a1d9 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 @@ -33,7 +33,7 @@ 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.{AnalysisException, SQLContext} 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,7 +45,8 @@ import org.apache.spark.util.Utils private[sql] class HiveSessionCatalog( externalCatalog: HiveExternalCatalog, client: HiveClient, - context: HiveContext, + context: SQLContext, + sessionState: HiveSessionState, functionResourceLoader: FunctionResourceLoader, functionRegistry: FunctionRegistry, conf: SQLConf) @@ -75,7 +76,7 @@ private[sql] class HiveSessionCatalog( // ---------------------------------------------------------------- override def getDefaultDBPath(db: String): String = { - val defaultPath = context.hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) + val defaultPath = sessionState.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(client, context, sessionState) val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions 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 bc28b55d06d9e..20d60108faec2 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,6 +17,9 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.conf.HiveConf +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 @@ -30,30 +33,54 @@ 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) { +// TODO(andrew): once we have SparkSession just pass that in here instead of the context +private[hive] class HiveSessionState(ctx: SQLContext, sharedState: HiveSharedState) + extends SessionState(ctx) { + + self => /** * A Hive client used for execution. */ - val executionHive: HiveClientImpl = ctx.hiveSharedState.executionHive.newSession() + val executionHive: HiveClientImpl = sharedState.executionHive.newSession() /** * A Hive client used for interacting with the metastore. */ - val metadataHive: HiveClient = ctx.hiveSharedState.metadataHive.newSession() + val metadataHive: HiveClient = sharedState.metadataHive.newSession() + + /** + * A Hive helper class for substituting variables in a SQL statement. + */ + 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. + */ + val hiveconf: HiveConf = { + val c = executionHive.conf + conf.setConf(c.getAllProperties) + c + } + /** * Internal catalog for managing table and database states. */ override lazy val catalog = { new HiveSessionCatalog( - ctx.hiveCatalog, - ctx.metadataHive, + sharedState.externalCatalog, + sharedState.metadataHive, ctx, + self, ctx.functionResourceLoader, functionRegistry, conf) @@ -80,7 +107,7 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) /** * Parser for HiveQl query texts. */ - override lazy val sqlParser: ParserInterface = HiveSqlParser + override lazy val sqlParser: ParserInterface = new HiveSqlParser(substitutor, hiveconf) /** * Planner that takes into account Hive-specific strategies. @@ -88,13 +115,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, @@ -114,4 +142,60 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) } } + /** + * 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 = { + 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. + */ + protected[sql] 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. + */ + protected[sql] 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). + */ + protected[sql] def convertCTAS: Boolean = { + conf.getConf(HiveContext.CONVERT_CTAS) + } + + /* + * hive thrift server use background spark sql thread pool to execute sql queries + */ + protected[hive] def hiveThriftServerAsync: Boolean = { + conf.getConf(HiveContext.HIVE_THRIFT_SERVER_ASYNC) + } + + protected[hive] def hiveThriftServerSingleSession: Boolean = { + ctx.sparkContext.conf.getBoolean( + "spark.sql.hive.thriftServer.singleSession", defaultValue = false) + } + } 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 010361a32eb34..bbdcc8c6c2fff 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( 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 54afe9c2a3550..6a20d7c25b682 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,12 +73,12 @@ class HadoopTableReader( private val _minSplitsPerRDD = if (sc.sparkContext.isLocal) { 0 // will splitted based on block by default. } else { - math.max(sc.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( @@ -162,7 +163,7 @@ class HadoopTableReader( case (partition, partDeserializer) => def updateExistPathSetByPathPattern(pathPatternStr: String) { val pathPattern = new Path(pathPatternStr) - val fs = pathPattern.getFileSystem(sc.hiveconf) + val fs = pathPattern.getFileSystem(hiveconf) val matches = fs.globStatus(pathPattern) matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString) } @@ -259,7 +260,7 @@ class HadoopTableReader( private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { filterOpt match { case Some(filter) => - val fs = path.getFileSystem(sc.hiveconf) + val fs = path.getFileSystem(hiveconf) val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) filteredFiles.mkString(",") case None => path.toString 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 a97b65e27bc59..4ff02cdbd0b39 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,36 +38,28 @@ import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper /** * Concrete parser for HiveQl statements. */ -object HiveSqlParser extends AbstractSqlParser { - val astBuilder = new HiveSqlAstBuilder +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)) } } /** * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier. */ -class HiveSqlAstBuilder extends SparkSqlAstBuilder { +class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { import ParserUtils._ - /** - * Get the current Hive Configuration. - */ - private[this] def hiveConf: HiveConf = { - var ss = SessionState.get() - // SessionState is lazy initialization, it can be null here - if (ss == null) { - val original = Thread.currentThread().getContextClassLoader - val conf = new HiveConf(classOf[SessionState]) - conf.setClassLoader(original) - ss = new SessionState(conf) - SessionState.start(ss) - } - ss.getConf - } - /** * Pass a command to Hive using a [[HiveNativeCommand]]. */ 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 235b80b7c697c..9a834660f953f 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.hiveconf) + private[this] val hiveExtraConf = new HiveConf(hiveconf) // append columns ids and names before broadcast addColumnMetadataToConf(hiveExtraConf) 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 3566526561b2f..2f7cec354d84f 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.hiveconf) + private val serializedHiveConf = new SerializableConfiguration(hiveconf) protected override def doExecute(): RDD[InternalRow] = { def processIterator(inputIterator: Iterator[InternalRow]): Iterator[InternalRow] = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index d56d36fe32e77..211650464aeb9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -136,13 +136,13 @@ class TestHiveContext private[hive]( // Override so we can intercept relative paths and rewrite them to point at hive. override def runSqlHive(sql: String): Seq[String] = - super.runSqlHive(rewritePaths(substitutor.substitute(this.hiveconf, sql))) + super.runSqlHive(rewritePaths(sessionState.substitutor.substitute(this.hiveconf, sql))) override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) @transient - protected[sql] override lazy val sessionState = new HiveSessionState(this) { + protected[sql] override lazy val sessionState = new HiveSessionState(this, testHiveSharedState) { override lazy val conf: SQLConf = { new SQLConf { clear() @@ -215,7 +215,7 @@ class TestHiveContext private[hive]( */ class QueryExecution(logicalPlan: LogicalPlan) extends super.QueryExecution(logicalPlan) { - def this(sql: String) = this(parseSql(sql)) + def this(sql: String) = this(sessionState.sqlParser.parsePlan(sql)) override lazy val analyzed = { val describedTables = logical match { case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index d9664680f4a11..2e3d5d8014545 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.hive.execution.HiveSqlParser import org.apache.spark.sql.hive.test.TestHiveSingleton class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterEach { @@ -131,7 +130,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = HiveSqlParser.parsePlan(query) + def ast = hiveContext.sessionState.sqlParser.parsePlan(query) def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 110c6d19d89ba..484cf528e6db7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -30,10 +30,11 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Generate, ScriptTransformation} import org.apache.spark.sql.execution.command.{CreateTable, CreateTableLike} -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, HiveSqlParser} +import org.apache.spark.sql.hive.execution.HiveNativeCommand +import org.apache.spark.sql.hive.test.TestHive class HiveDDLCommandSuite extends PlanTest { - val parser = HiveSqlParser + val parser = TestHive.sessionState.sqlParser private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { parser.parsePlan(sql).collect { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 05318f51af01e..d14c72b34bc9a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -31,7 +31,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { - val parsed = HiveSqlParser.parsePlan(analyzeCommand) + val parsed = hiveContext.sessionState.sqlParser.parsePlan(analyzeCommand) val operators = parsed.collect { case a: AnalyzeTable => a case o => o diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index f5cd73d45ed75..c09403b55f6fc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -30,9 +30,9 @@ class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { conf.set("spark.ui.enabled", "false") val ts = new TestHiveContext(new SparkContext("local", s"TestSQLContext$i", conf)) - ts.executeSql("SHOW TABLES").toRdd.collect() - ts.executeSql("SELECT * FROM src").toRdd.collect() - ts.executeSql("SHOW TABLES").toRdd.collect() + ts.executePlan(ts.sessionState.sqlParser.parsePlan("SHOW TABLES")).toRdd.collect() + ts.executePlan(ts.sessionState.sqlParser.parsePlan("SELECT * FROM src")).toRdd.collect() + ts.executePlan(ts.sessionState.sqlParser.parsePlan("SHOW TABLES")).toRdd.collect() } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 5ce16be4dc059..bceeb5002687d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -350,7 +350,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - val originalConf = convertCTAS + val originalConf = sessionState.convertCTAS setConf(HiveContext.CONVERT_CTAS, true) @@ -732,7 +732,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // generates an invalid query plan. val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) read.json(rdd).registerTempTable("data") - val originalConf = convertCTAS + val originalConf = sessionState.convertCTAS setConf(HiveContext.CONVERT_CTAS, false) try { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 8f163f27c94cf..8e560b3142c1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -58,7 +58,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = noSerdeIOSchema - )(hiveContext), + )(hiveContext.hiveconf), rowsDf.collect()) } @@ -72,7 +72,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = serdeIOSchema - )(hiveContext), + )(hiveContext.hiveconf), rowsDf.collect()) } @@ -87,7 +87,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = noSerdeIOSchema - )(hiveContext), + )(hiveContext.hiveconf), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) @@ -104,7 +104,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = serdeIOSchema - )(hiveContext), + )(hiveContext.hiveconf), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) From 54046d66f68ad06d78bbda73207651c1f29518ab Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 13:57:33 -0700 Subject: [PATCH 03/27] Move QueryExecution out of HiveContext --- .../hive/thriftserver/SparkSQLDriver.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 41 +----------- .../spark/sql/hive/HiveQueryExecution.scala | 67 +++++++++++++++++++ .../apache/spark/sql/hive/test/TestHive.scala | 2 +- 4 files changed, 71 insertions(+), 43 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.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 b8bc8ea44dc84..c90bc8af50a0d 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) { 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 2545bfcd7858c..1e8bd447c6a96 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 @@ -97,8 +97,7 @@ class HiveContext private[hive]( defaultOverrides() - override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = - new this.QueryExecution(plan) + override protected[sql] def executePlan(plan: LogicalPlan) = new HiveQueryExecution(self, plan) /** * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, @@ -171,44 +170,6 @@ class HiveContext private[hive]( 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 => "" - case _: SetCommand => "" - case _ => super.simpleString - } - } - protected[sql] override def addJar(path: String): Unit = { // Add jar to Hive and classloader executionHive.addJar(path) 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 0000000000000..c2c5079fb82a5 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala @@ -0,0 +1,67 @@ +/* + * 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[sql] 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. + // TODO(andrew): this might be a problem. This uses the wrong session state under the hood. + 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 => "" + case _: SetCommand => "" + case _ => super.simpleString + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 211650464aeb9..d5d392cf1df27 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -214,7 +214,7 @@ class TestHiveContext private[hive]( * Override QueryExecution with special debug workflow. */ class QueryExecution(logicalPlan: LogicalPlan) - extends super.QueryExecution(logicalPlan) { + extends HiveQueryExecution(self, logicalPlan) { def this(sql: String) = this(sessionState.sqlParser.parsePlan(sql)) override lazy val analyzed = { val describedTables = logical match { From 83b3f7094290613e22713b96a41b937cf9200217 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 14:11:54 -0700 Subject: [PATCH 04/27] Minor cleanup --- .../org/apache/spark/sql/hive/HiveContext.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) 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 b0b26ac2c026b..b142014c38ee3 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 @@ -41,10 +41,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.sql._ 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.hive.execution.AnalyzeTable +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -146,12 +145,11 @@ class HiveContext private[hive]( setConf(entry.key, entry.stringConverter(value)) } - 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()) { + 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) From b33514c241fb56514cd734caa6741fa8ba1cfb4f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 15:53:17 -0700 Subject: [PATCH 05/27] Implement SparkSession and use it to track state Now both shared state and session state is tracked in SparkSession and we use reflection to instantiate them. After this commit SQLContext and HiveContext are just wrappers for SparkSession. --- .../org/apache/spark/sql/SQLContext.scala | 32 +++---- .../org/apache/spark/sql/SparkSession.scala | 83 ++++++++++++++++ .../spark/sql/internal/SessionState.scala | 12 ++- .../spark/sql/test/TestSQLContext.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 21 ++-- .../spark/sql/hive/HiveSessionState.scala | 8 +- .../apache/spark/sql/hive/test/TestHive.scala | 95 ++++++++++++------- 7 files changed, 186 insertions(+), 67 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/SparkSession.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 c51b1a71ff387..e03b8d07b83f5 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 @@ -186,12 +188,6 @@ 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 executePlan(plan: LogicalPlan) = new QueryExecution(this, plan) /** 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 0000000000000..c97a5d5588391 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -0,0 +1,83 @@ +/* + * 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.util.control.NonFatal + +import org.apache.spark.SparkContext +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(SparkSession.DEFAULT_SHARED_STATE_CLASS_NAME, sparkContext)) + } + + @transient + protected[sql] lazy val sessionState: SessionState = { + SparkSession.reflect( + SparkSession.DEFAULT_SESSION_STATE_CLASS_NAME, + new SQLContext(self, isRootContext = false)) + } + +} + + +private object SparkSession { + + private val DEFAULT_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" + private val DEFAULT_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" + + /** + * 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): T = { + try { + val clazz = Utils.classForName(className) + val ctor = clazz.getDeclaredConstructor(ctorArg.getClass) + ctor.newInstance(ctorArg).asInstanceOf[T] + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Error while instantiating '$className':", e) + } + } + +} 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 d404a7c0aef59..928f980fe8189 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 @@ -34,10 +34,19 @@ private[sql] class SessionState(ctx: SQLContext) { // Note: These are all lazy vals because they depend on each other (e.g. conf) and we // want subclasses to override some of the fields. Otherwise, we would get a lot of NPEs. + protected def newConf(): SQLConf = new SQLConf + /** * SQL-specific key-value configurations. */ - lazy val conf = new SQLConf + final lazy val conf: SQLConf = { + val _conf = newConf() + // Extract `spark.sql.*` entries and put it in our SQLConf. + // Subclasses may additionally set these entries in other confs. + _conf.setConf(SQLContext.getSQLProperties(ctx.sparkContext.getConf)) + _conf + } + lazy val experimentalMethods = new ExperimentalMethods @@ -102,4 +111,3 @@ private[sql] class SessionState(ctx: SQLContext) { */ lazy val continuousQueryManager: ContinuousQueryManager = new ContinuousQueryManager(ctx) } - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala index 7ab79b12ce246..6efe662d31dbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -37,7 +37,7 @@ private[sql] class TestSQLContext(sc: SparkContext) extends SQLContext(sc) { sel @transient protected[sql] override lazy val sessionState: SessionState = new SessionState(self) { - override lazy val conf: SQLConf = { + override def newConf(): SQLConf = { new SQLConf { clear() override def clear(): Unit = { 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 b142014c38ee3..0cc0f2cecafb8 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 @@ -55,14 +55,14 @@ 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(sc), true) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -75,13 +75,18 @@ class HiveContext private[hive]( * and Hive client (both of execution and metadata) with existing HiveContext. */ override def newSession(): HiveContext = { - new HiveContext(hiveSharedState, isRootContext = false) + new HiveContext(sparkSession.newSession(), isRootContext = false) } - @transient - protected[sql] override lazy val sessionState = new HiveSessionState(self, hiveSharedState) + protected[sql] override def sessionState = { + sparkSession.sessionState.asInstanceOf[HiveSessionState] + } + + protected[sql] override def sharedState = { + sparkSession.sharedState.asInstanceOf[HiveSharedState] + } - protected[hive] def hiveCatalog: HiveExternalCatalog = hiveSharedState.externalCatalog + protected[hive] def hiveCatalog: HiveExternalCatalog = sharedState.externalCatalog protected[hive] def executionHive: HiveClientImpl = sessionState.executionHive protected[hive] def metadataHive: HiveClient = sessionState.metadataHive protected[hive] def hiveconf: HiveConf = sessionState.hiveconf 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 20d60108faec2..ebd3e0a15aa69 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 @@ -33,12 +33,12 @@ import org.apache.spark.sql.internal.{SessionState, SQLConf} /** * A class that holds all session-specific state in a given [[HiveContext]]. */ -// TODO(andrew): once we have SparkSession just pass that in here instead of the context -private[hive] class HiveSessionState(ctx: SQLContext, sharedState: HiveSharedState) - extends SessionState(ctx) { +private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) { self => + private val sharedState: HiveSharedState = ctx.sharedState.asInstanceOf[HiveSharedState] + /** * A Hive client used for execution. */ @@ -54,7 +54,7 @@ private[hive] class HiveSessionState(ctx: SQLContext, sharedState: HiveSharedSta */ val substitutor = new VariableSubstitution - override lazy val conf: SQLConf = new SQLConf { + override def newConf(): SQLConf = new SQLConf { override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index d5d392cf1df27..6858c9f592218 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -32,16 +32,14 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.command.CacheTableCommand -import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.client.{HiveClient, HiveClientImpl} +import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{ShutdownHookManager, Utils} @@ -72,12 +70,12 @@ object TestHive * test cases that rely on TestHive must be serialized. */ class TestHiveContext private[hive]( - testHiveSharedState: TestHiveSharedState, + sparkSession: SparkSession, val warehousePath: File, val scratchDirPath: File, metastoreTemporaryConf: Map[String, String], isRootContext: Boolean) - extends HiveContext(testHiveSharedState, isRootContext) { self => + extends HiveContext(sparkSession, isRootContext) { self => private def this( sc: SparkContext, @@ -85,7 +83,7 @@ class TestHiveContext private[hive]( scratchDirPath: File, metastoreTemporaryConf: Map[String, String]) { this( - new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf), + new TestHiveSparkSession(sc, warehousePath, scratchDirPath, metastoreTemporaryConf), warehousePath, scratchDirPath, metastoreTemporaryConf, @@ -102,13 +100,21 @@ class TestHiveContext private[hive]( override def newSession(): HiveContext = { new TestHiveContext( - testHiveSharedState, + sparkSession.newSession(), warehousePath, scratchDirPath, metastoreTemporaryConf, isRootContext = false) } + protected[sql] override def sessionState = { + sparkSession.sessionState.asInstanceOf[TestHiveSessionState] + } + + protected[sql] override def sharedState = { + sparkSession.sharedState.asInstanceOf[TestHiveSharedState] + } + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") @@ -141,32 +147,6 @@ class TestHiveContext private[hive]( override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) - @transient - protected[sql] override lazy val sessionState = new HiveSessionState(this, testHiveSharedState) { - override lazy val conf: SQLConf = { - new SQLConf { - clear() - override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) - override def clear(): Unit = { - super.clear() - TestHiveContext.overrideConfs.map { - case (key, value) => setConfString(key, value) - } - } - } - } - - override lazy val functionRegistry = { - // We use TestHiveFunctionRegistry at here to track functions that have been explicitly - // unregistered (through TestHiveFunctionRegistry.unregisterFunction method). - val fr = new TestHiveFunctionRegistry - org.apache.spark.sql.catalyst.analysis.FunctionRegistry.expressions.foreach { - case (name, (info, builder)) => fr.registerFunction(name, info, builder) - } - fr - } - } - /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists @@ -489,6 +469,24 @@ class TestHiveContext private[hive]( } + +private[hive] class TestHiveSparkSession( + sc: SparkContext, + warehousePath: File, + scratchDirPath: File, + metastoreTemporaryConf: Map[String, String]) + extends SparkSession(sc) { + + self => + + override lazy val sharedState = + new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf) + + override lazy val sessionState = + new TestHiveSessionState(new SQLContext(self)) +} + + private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { private val removedFunctions = @@ -521,6 +519,35 @@ private[hive] class TestHiveSharedState( } +protected[hive] class TestHiveSessionState(sqlContext: SQLContext) + extends HiveSessionState(sqlContext) { + + override def newConf(): SQLConf = { + new SQLConf { + clear() + override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) + override def clear(): Unit = { + super.clear() + TestHiveContext.overrideConfs.map { + case (key, value) => setConfString(key, value) + } + } + } + } + + override lazy val functionRegistry = { + // We use TestHiveFunctionRegistry at here to track functions that have been explicitly + // unregistered (through TestHiveFunctionRegistry.unregisterFunction method). + val fr = new TestHiveFunctionRegistry + org.apache.spark.sql.catalyst.analysis.FunctionRegistry.expressions.foreach { + case (name, (info, builder)) => fr.registerFunction(name, info, builder) + } + fr + } + +} + + private[hive] object TestHiveContext { /** From 6b808aa411ec099b8d856b16ead1d7f9d7c342c0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 16:29:20 -0700 Subject: [PATCH 06/27] Clean up some TODO's and bad signatures --- .../org/apache/spark/sql/SQLContext.scala | 4 ++++ .../apache/spark/sql/hive/HiveContext.scala | 4 ++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 19 ++++++++----------- .../spark/sql/hive/HiveQueryExecution.scala | 1 - .../spark/sql/hive/HiveSessionCatalog.scala | 8 ++++---- .../spark/sql/hive/HiveSessionState.scala | 4 ++-- .../apache/spark/sql/hive/test/TestHive.scala | 6 +++--- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../hive/execution/ConcurrentHiveSuite.scala | 6 +++--- 10 files changed, 28 insertions(+), 28 deletions(-) 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 587a880f4004f..734a3b4ca0dee 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 @@ -188,6 +188,10 @@ class SQLContext private[sql]( */ def getAllConfs: immutable.Map[String, String] = conf.getAllConfs + 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) /** 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 4eaf492177eb4..4863f393c7ce7 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 @@ -78,11 +78,11 @@ class HiveContext private[hive]( new HiveContext(sparkSession.newSession(), isRootContext = false) } - protected[sql] override def sessionState = { + protected[sql] override def sessionState: HiveSessionState = { sparkSession.sessionState.asInstanceOf[HiveSessionState] } - protected[sql] override def sharedState = { + protected[sql] override def sharedState: HiveSharedState = { sparkSession.sharedState.asInstanceOf[HiveSharedState] } 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 9bf1c7bc9daff..72fd7ff617182 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 @@ -116,19 +116,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]]. */ -// TODO(andrew): once we have SparkSession just pass that in here instead of the context -private[hive] class HiveMetastoreCatalog( - val client: HiveClient, - hive: SQLContext, - sessionState: HiveSessionState) - extends Logging { - +private[hive] class HiveMetastoreCatalog(hive: SQLContext) extends Logging { val conf = hive.conf + val sessionState = hive.sessionState.asInstanceOf[HiveSessionState] + val client = hive.sharedState.asInstanceOf[HiveSharedState].metadataHive + 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 = sessionState.catalog.getCurrentDatabase + private def getCurrentDatabase: String = hive.sessionState.catalog.getCurrentDatabase def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = { QualifiedTableName( @@ -301,7 +298,7 @@ private[hive] class HiveMetastoreCatalog( CatalogTableType.MANAGED_TABLE } - val maybeSerDe = HiveSerDe.sourceToSerDe(provider, sessionState.hiveconf) + val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hiveconf) val dataSource = DataSource( hive, @@ -446,9 +443,9 @@ private[hive] class HiveMetastoreCatalog( // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." case None => - SubqueryAlias(table.identifier.table, sessionState.sqlParser.parsePlan(viewText)) + SubqueryAlias(table.identifier.table, hive.parseSql(viewText)) case Some(aliasText) => - SubqueryAlias(aliasText, sessionState.sqlParser.parsePlan(viewText)) + SubqueryAlias(aliasText, hive.parseSql(viewText)) } } else { MetastoreRelation( 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 index c2c5079fb82a5..66d2d84793416 100644 --- 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 @@ -38,7 +38,6 @@ protected[sql] class HiveQueryExecution(ctx: SQLContext, logicalPlan: LogicalPla 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. - // TODO(andrew): this might be a problem. This uses the wrong session state under the hood. desc.run(ctx).map { case Row(name: String, dataType: String, comment) => Seq(name, dataType, 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 9f3f4ce41a1d9..3adaca66bcb41 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 @@ -46,10 +46,10 @@ private[sql] class HiveSessionCatalog( externalCatalog: HiveExternalCatalog, client: HiveClient, context: SQLContext, - sessionState: HiveSessionState, functionResourceLoader: FunctionResourceLoader, functionRegistry: FunctionRegistry, - conf: SQLConf) + conf: SQLConf, + hiveconf: HiveConf) extends SessionCatalog(externalCatalog, functionResourceLoader, functionRegistry, conf) { override def setCurrentDatabase(db: String): Unit = { @@ -76,7 +76,7 @@ private[sql] class HiveSessionCatalog( // ---------------------------------------------------------------- override def getDefaultDBPath(db: String): String = { - val defaultPath = sessionState.hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) + val defaultPath = hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) new Path(new Path(defaultPath), db + ".db").toString } @@ -84,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, sessionState) + private val metastoreCatalog = new HiveMetastoreCatalog(context) val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions 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 ebd3e0a15aa69..1f6d614e463b7 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 @@ -80,10 +80,10 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) sharedState.externalCatalog, sharedState.metadataHive, ctx, - self, ctx.functionResourceLoader, functionRegistry, - conf) + conf, + hiveconf) } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 0ec18bc2ad967..040af3d202896 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -107,11 +107,11 @@ class TestHiveContext private[hive]( isRootContext = false) } - protected[sql] override def sessionState = { + protected[sql] override def sessionState: TestHiveSessionState = { sparkSession.sessionState.asInstanceOf[TestHiveSessionState] } - protected[sql] override def sharedState = { + protected[sql] override def sharedState: TestHiveSharedState = { sparkSession.sharedState.asInstanceOf[TestHiveSharedState] } @@ -195,7 +195,7 @@ class TestHiveContext private[hive]( */ class QueryExecution(logicalPlan: LogicalPlan) extends HiveQueryExecution(self, logicalPlan) { - def this(sql: String) = this(sessionState.sqlParser.parsePlan(sql)) + def this(sql: String) = this(parseSql(sql)) override lazy val analyzed = { val describedTables = logical match { case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index 2e3d5d8014545..61910b8e6b51d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -130,7 +130,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = hiveContext.sessionState.sqlParser.parsePlan(query) + def ast = hiveContext.parseSql(query) def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index d14c72b34bc9a..f411c3b34174b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -31,7 +31,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { - val parsed = hiveContext.sessionState.sqlParser.parsePlan(analyzeCommand) + val parsed = hiveContext.parseSql(analyzeCommand) val operators = parsed.collect { case a: AnalyzeTable => a case o => o diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index c09403b55f6fc..f5cd73d45ed75 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -30,9 +30,9 @@ class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { conf.set("spark.ui.enabled", "false") val ts = new TestHiveContext(new SparkContext("local", s"TestSQLContext$i", conf)) - ts.executePlan(ts.sessionState.sqlParser.parsePlan("SHOW TABLES")).toRdd.collect() - ts.executePlan(ts.sessionState.sqlParser.parsePlan("SELECT * FROM src")).toRdd.collect() - ts.executePlan(ts.sessionState.sqlParser.parsePlan("SHOW TABLES")).toRdd.collect() + ts.executeSql("SHOW TABLES").toRdd.collect() + ts.executeSql("SELECT * FROM src").toRdd.collect() + ts.executeSql("SHOW TABLES").toRdd.collect() } } } From 5198955b26c9c6d070a5ffe7292408ed9db2a525 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 17:47:26 -0700 Subject: [PATCH 07/27] Move the bulk of HiveContext into SessionCatalog --- .../org/apache/spark/sql/SQLContext.scala | 10 +- .../spark/sql/internal/SessionState.scala | 54 +++++++++-- .../spark/sql/test/TestSQLContext.scala | 2 +- .../hive/thriftserver/SparkSQLDriver.scala | 3 +- .../apache/spark/sql/hive/HiveContext.scala | 92 ------------------- .../spark/sql/hive/HiveSessionState.scala | 84 +++++++++++++++-- .../hive/execution/HiveNativeCommand.scala | 8 +- .../spark/sql/hive/execution/commands.scala | 2 +- .../apache/spark/sql/hive/test/TestHive.scala | 92 +++++++++++-------- .../sql/hive/HiveMetastoreCatalogSuite.scala | 6 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 20 ++-- .../spark/sql/hive/MultiDatabaseSuite.scala | 4 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 4 +- .../execution/ScriptTransformationSuite.scala | 8 +- .../apache/spark/sql/hive/parquetSuites.scala | 6 +- 16 files changed, 213 insertions(+), 184 deletions(-) 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 734a3b4ca0dee..d093e38f2c1fe 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 @@ -134,7 +134,7 @@ 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) @@ -145,7 +145,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. @@ -192,7 +192,9 @@ class SQLContext private[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 @@ -764,7 +766,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) } } /** 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 928f980fe8189..dacf67c25e596 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,21 @@ package org.apache.spark.sql.internal +import java.util.Properties + +import scala.collection.JavaConverters._ + 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]]. */ @@ -34,19 +40,13 @@ private[sql] class SessionState(ctx: SQLContext) { // Note: These are all lazy vals because they depend on each other (e.g. conf) and we // want subclasses to override some of the fields. Otherwise, we would get a lot of NPEs. - protected def newConf(): SQLConf = new SQLConf - /** * SQL-specific key-value configurations. */ - final lazy val conf: SQLConf = { - val _conf = newConf() - // Extract `spark.sql.*` entries and put it in our SQLConf. - // Subclasses may additionally set these entries in other confs. - _conf.setConf(SQLContext.getSQLProperties(ctx.sparkContext.getConf)) - _conf - } + 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 @@ -110,4 +110,40 @@ 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) } + } + + 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 + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala index 6efe662d31dbd..7ab79b12ce246 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -37,7 +37,7 @@ private[sql] class TestSQLContext(sc: SparkContext) extends SQLContext(sc) { sel @transient protected[sql] override lazy val sessionState: SessionState = new SessionState(self) { - override def newConf(): SQLConf = { + override lazy val conf: SQLConf = { new SQLConf { clear() override def clear(): Unit = { 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 c90bc8af50a0d..7e8eada5adb4f 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 @@ -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) 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 4863f393c7ce7..b40d31f1e30eb 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 @@ -38,11 +38,8 @@ 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.sql._ -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.AnalyzeTable import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.types._ @@ -91,95 +88,6 @@ class HiveContext private[hive]( protected[hive] def metadataHive: HiveClient = sessionState.metadataHive protected[hive] def hiveconf: HiveConf = sessionState.hiveconf - /** - * 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() - - override protected[sql] def executePlan(plan: LogicalPlan) = new HiveQueryExecution(self, 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) - } - - override private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = { - setConf(entry.key, entry.stringConverter(value)) - } - - protected[hive] def runSqlHive(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) - } - } - - /** - * 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) } - } - - 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) - } } 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 1f6d614e463b7..2706cc356b749 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,16 +17,20 @@ 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} @@ -54,10 +58,12 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) */ val substitutor = new VariableSubstitution - override def newConf(): SQLConf = new SQLConf { + override lazy val conf: SQLConf = new SQLConf { override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) } + setDefaultOverrideConfs() + /** * SQLConf and HiveConf contracts: * @@ -142,12 +148,70 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) } } + + // ------------------------------------------------------ + // Helper methods, partially leftover from pre-2.0 days + // ------------------------------------------------------ + + override def executePlan(plan: LogicalPlan) = new HiveQueryExecution(ctx, plan) + + /** + * Overrides default Hive configurations to avoid breaking changes to Spark SQL users. + * - allow SQL11 keywords to be used as identifiers + */ + private[sql] 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. + */ + // TODO: not used at the moment + 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. */ - protected[sql] def convertMetastoreParquet: Boolean = { + def convertMetastoreParquet: Boolean = { conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET) } @@ -157,7 +221,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true. */ - protected[sql] def convertMetastoreParquetWithSchemaMerging: Boolean = { + def convertMetastoreParquetWithSchemaMerging: Boolean = { conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING) } @@ -166,7 +230,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * are automatically converted to use the Spark SQL ORC table scan, instead of the Hive * SerDe. */ - protected[sql] def convertMetastoreOrc: Boolean = { + def convertMetastoreOrc: Boolean = { conf.getConf(HiveContext.CONVERT_METASTORE_ORC) } @@ -182,18 +246,18 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * - 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 = { + def convertCTAS: Boolean = { conf.getConf(HiveContext.CONVERT_CTAS) } - /* - * hive thrift server use background spark sql thread pool to execute sql queries + /** + * When true, Hive Thrift server will execute SQL queries asynchronously using a thread pool." */ - protected[hive] def hiveThriftServerAsync: Boolean = { + def hiveThriftServerAsync: Boolean = { conf.getConf(HiveContext.HIVE_THRIFT_SERVER_ASYNC) } - protected[hive] def hiveThriftServerSingleSession: Boolean = { + def hiveThriftServerSingleSession: Boolean = { ctx.sparkContext.conf.getBoolean( "spark.sql.hive.thriftServer.singleSession", defaultValue = false) } 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 9bb971992d0d1..8c1f4a8dc5139 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(_)) + } + } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 5ef502afa5e67..19c0077d2c848 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -145,7 +145,7 @@ case class AddFile(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] - hiveContext.runSqlHive(s"ADD FILE $path") + hiveContext.sessionState.runNativeSql(s"ADD FILE $path") hiveContext.sparkContext.addFile(path) Seq.empty[Row] } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 040af3d202896..13f0b8b0b4aca 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -137,15 +137,13 @@ class TestHiveContext private[hive]( /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") - /** The location of the hive source code. */ - lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") - - // Override so we can intercept relative paths and rewrite them to point at hive. - override def runSqlHive(sql: String): Seq[String] = - super.runSqlHive(rewritePaths(sessionState.substitutor.substitute(hiveconf, sql))) - override def executePlan(plan: LogicalPlan): this.QueryExecution = - new this.QueryExecution(plan) + /** The location of the hive source code. */ + lazy val hiveDevHome = { + val f = envVarToFile("HIVE_DEV_HOME") + sessionState.setHiveDevHome(f) + f + } /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking @@ -155,29 +153,20 @@ class TestHiveContext private[hive]( Option(System.getenv(envVar)).map(new File(_)) } - /** - * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the - * hive test cases assume the system is set up. - */ - private def rewritePaths(cmd: String): String = - if (cmd.toUpperCase contains "LOAD DATA") { - val testDataLocation = - hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) - cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") - } else { - cmd - } - val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() ShutdownHookManager.registerShutdownDeleteDir(hiveFilesTemp) - val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { - new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) - } else { - new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + - File.separator + "resources") + val inRepoTests: File = { + val f = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { + new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) + } else { + new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + + File.separator + "resources") + } + sessionState.setInRepoTests(f) + f } def getHiveFile(path: String): File = { @@ -246,19 +235,19 @@ class TestHiveContext private[hive]( "CREATE TABLE src1 (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("srcpart", () => { - runSqlHive( + sessionState.runNativeSql( "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { - runSqlHive( + sessionState.runNativeSql( s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') """.stripMargin) } }), TestTable("srcpart1", () => { - runSqlHive("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") + sessionState.runNativeSql("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { - runSqlHive( + sessionState.runNativeSql( s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') """.stripMargin) @@ -269,7 +258,7 @@ class TestHiveContext private[hive]( import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} import org.apache.thrift.protocol.TBinaryProtocol - runSqlHive( + sessionState.runNativeSql( s""" |CREATE TABLE src_thrift(fake INT) |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' @@ -282,7 +271,7 @@ class TestHiveContext private[hive]( |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' """.stripMargin) - runSqlHive( + sessionState.runNativeSql( s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") }), TestTable("serdeins", @@ -448,17 +437,17 @@ class TestHiveContext private[hive]( metadataHive.runSqlHive("RESET") // For some reason, RESET does not reset the following variables... // https://issues.apache.org/jira/browse/HIVE-9004 - runSqlHive("set hive.table.parameters.default=") - runSqlHive("set datanucleus.cache.collections=true") - runSqlHive("set datanucleus.cache.collections.lazy=true") + sessionState.runNativeSql("set hive.table.parameters.default=") + sessionState.runNativeSql("set datanucleus.cache.collections=true") + sessionState.runNativeSql("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. - runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") + sessionState.runNativeSql("set hive.metastore.partition.name.whitelist.pattern=.*") // In case a test changed any of these values, restore all the original ones here. TestHiveContext.hiveClientConfigurations( sessionState.hiveconf, warehousePath, scratchDirPath, metastoreTemporaryConf) .foreach { case (k, v) => metadataHive.runSqlHive(s"SET $k=$v") } - defaultOverrides() + sessionState.setDefaultOverrideConfs() sessionState.catalog.setCurrentDatabase("default") } catch { @@ -522,7 +511,15 @@ private[hive] class TestHiveSharedState( protected[hive] class TestHiveSessionState(sqlContext: SQLContext) extends HiveSessionState(sqlContext) { - override def newConf(): SQLConf = { + // Hack alert: These will be set in TestHiveContext constructor. Due to initialization order + // constraints we can't pass them in through the session state constructor. Sorry to build on + // top of this mess! + private var hiveDevHome: Option[File] = None + private var inRepoTests: File = _ + def setHiveDevHome(f: Option[File]): Unit = { hiveDevHome = f } + def setInRepoTests(f: File): Unit = { inRepoTests = f } + + override lazy val conf: SQLConf = { new SQLConf { clear() override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) @@ -545,6 +542,25 @@ protected[hive] class TestHiveSessionState(sqlContext: SQLContext) fr } + // Override so we can intercept relative paths and rewrite them to point at hive. + override def runNativeSql(sql: String): Seq[String] = { + super.runNativeSql(rewritePaths(substitutor.substitute(hiveconf, sql))) + } + + /** + * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the + * hive test cases assume the system is set up. + */ + private def rewritePaths(cmd: String): String = { + if (cmd.toUpperCase contains "LOAD DATA") { + val testDataLocation = + hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) + cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") + } else { + cmd + } + } + } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 8648834f0d881..2a201c195f167 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -96,7 +96,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) - assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) + assert(sessionState.runNativeSql("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) } } @@ -129,7 +129,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) - assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) + assert(sessionState.runNativeSql("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) } } } @@ -159,7 +159,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq("int", "string")) checkAnswer(table("t"), Row(1, "val_1")) - assert(runSqlHive("SELECT * FROM t") === Seq("1\tval_1")) + assert(sessionState.runNativeSql("SELECT * FROM t") === Seq("1\tval_1")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 3c299daa778cc..1b7f82cbbe9c5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -254,13 +254,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) // Discard the cached relation. - invalidateTable("jsonTable") + sessionState.invalidateTable("jsonTable") checkAnswer( sql("SELECT * FROM jsonTable"), sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) - invalidateTable("jsonTable") + sessionState.invalidateTable("jsonTable") val expectedSchema = StructType(StructField("c_!@(3)", IntegerType, true) :: Nil) assert(expectedSchema === table("jsonTable").schema) @@ -348,7 +348,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) // Discard the cached relation. - invalidateTable("ctasJsonTable") + sessionState.invalidateTable("ctasJsonTable") // Schema should not be changed. assert(table("ctasJsonTable").schema === table("jsonTable").schema) @@ -423,7 +423,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"), (6 to 10).map(i => Row(i, s"str$i"))) - invalidateTable("savedJsonTable") + sessionState.invalidateTable("savedJsonTable") checkAnswer( sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"), @@ -621,7 +621,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("arrayInParquet") - refreshTable("arrayInParquet") + sessionState.refreshTable("arrayInParquet") checkAnswer( sql("SELECT a FROM arrayInParquet"), @@ -680,7 +680,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("mapInParquet") - refreshTable("mapInParquet") + sessionState.refreshTable("mapInParquet") checkAnswer( sql("SELECT a FROM mapInParquet"), @@ -708,7 +708,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv options = Map("path" -> tempDir.getCanonicalPath), isExternal = false) - invalidateTable("wide_schema") + sessionState.invalidateTable("wide_schema") val actualSchema = table("wide_schema").schema assert(schema === actualSchema) @@ -740,7 +740,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv hiveCatalog.createTable("default", hiveTable, ignoreIfExists = false) - invalidateTable(tableName) + sessionState.invalidateTable(tableName) val actualSchema = table(tableName).schema assert(schema === actualSchema) } @@ -752,7 +752,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv withTable(tableName) { df.write.format("parquet").partitionBy("d", "b").saveAsTable(tableName) - invalidateTable(tableName) + sessionState.invalidateTable(tableName) val metastoreTable = hiveCatalog.getTable("default", tableName) val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) @@ -787,7 +787,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .bucketBy(8, "d", "b") .sortBy("c") .saveAsTable(tableName) - invalidateTable(tableName) + sessionState.invalidateTable(tableName) val metastoreTable = hiveCatalog.getTable("default", tableName) val expectedBucketByColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val expectedSortByColumns = StructType(df.schema("c") :: Nil) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 3c003506efcb1..e84109a27e2b6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -216,7 +216,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql("ALTER TABLE t ADD PARTITION (p=2)") - hiveContext.refreshTable("t") + hiveContext.sessionState.refreshTable("t") checkAnswer( sqlContext.table("t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) @@ -248,7 +248,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql(s"ALTER TABLE $db.t ADD PARTITION (p=2)") - hiveContext.refreshTable(s"$db.t") + hiveContext.sessionState.refreshTable(s"$db.t") checkAnswer( sqlContext.table(s"$db.t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index f411c3b34174b..adc7af32ca03f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -116,7 +116,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Try to analyze a temp table sql("""SELECT * FROM src""").registerTempTable("tempTable") intercept[UnsupportedOperationException] { - hiveContext.analyze("tempTable") + hiveContext.sessionState.analyze("tempTable") } hiveContext.sessionState.catalog.dropTable( TableIdentifier("tempTable"), ignoreIfNotExists = true) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index e67fcbedc3364..384577f642b79 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -352,7 +352,7 @@ abstract class HiveComparisonTest case _: ExplainCommand => // No need to execute EXPLAIN queries as we don't check the output. Nil - case _ => TestHive.runSqlHive(queryString) + case _ => TestHive.sessionState.runNativeSql(queryString) } // We need to add a new line to non-empty answers so we can differentiate Seq() @@ -563,7 +563,7 @@ abstract class HiveComparisonTest // something must have gone seriously wrong. try { new TestHive.QueryExecution("SELECT key FROM src").stringResult() - TestHive.runSqlHive("SELECT key FROM src") + TestHive.sessionState.runNativeSql("SELECT key FROM src") } catch { case e: Exception => logError(s"FATAL ERROR: Canary query threw $e This implies that the " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 8e560b3142c1b..00b5c8dd41730 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -58,7 +58,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = noSerdeIOSchema - )(hiveContext.hiveconf), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } @@ -72,7 +72,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = serdeIOSchema - )(hiveContext.hiveconf), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } @@ -87,7 +87,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = noSerdeIOSchema - )(hiveContext.hiveconf), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) @@ -104,7 +104,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = serdeIOSchema - )(hiveContext.hiveconf), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index eac65d5720575..b3e6ca50a7336 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -462,7 +462,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { checkCached(tableIdentifier) // For insert into non-partitioned table, we will do the conversion, // so the converted test_insert_parquet should be cached. - invalidateTable("test_insert_parquet") + sessionState.invalidateTable("test_insert_parquet") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) sql( """ @@ -475,7 +475,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql("select * from test_insert_parquet"), sql("select a, b from jt").collect()) // Invalidate the cache. - invalidateTable("test_insert_parquet") + sessionState.invalidateTable("test_insert_parquet") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) // Create a partitioned table. @@ -525,7 +525,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { |select b, '2015-04-02', a FROM jt """.stripMargin).collect()) - invalidateTable("test_parquet_partitioned_cache_test") + sessionState.invalidateTable("test_parquet_partitioned_cache_test") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) dropTables("test_insert_parquet", "test_parquet_partitioned_cache_test") From d58c6afc332622dccde92f588327125c56766446 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 17:51:21 -0700 Subject: [PATCH 08/27] Remove more things from HiveContext --- .../SparkExecuteStatementOperation.scala | 2 +- .../spark/sql/hive/thriftserver/SparkSQLEnv.scala | 6 +++--- .../org/apache/spark/sql/hive/HiveContext.scala | 7 ------- .../sql/hive/execution/CreateViewAsSelect.scala | 4 ++-- .../sql/hive/execution/InsertIntoHiveTable.scala | 2 +- .../apache/spark/sql/hive/execution/commands.scala | 2 +- .../org/apache/spark/sql/hive/test/TestHive.scala | 8 ++++---- .../apache/spark/sql/hive/HiveContextSuite.scala | 3 ++- .../spark/sql/hive/MetastoreDataSourcesSuite.scala | 13 ++++++------- .../apache/spark/sql/hive/MultiDatabaseSuite.scala | 5 +++-- 10 files changed, 23 insertions(+), 29 deletions(-) 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 673a293ce2601..d89c3b4ab2d1c 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( 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 ae1d737b58adc..2679ac1854bb8 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) 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 b40d31f1e30eb..2ba1e3dc32afa 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 @@ -64,8 +64,6 @@ class HiveContext private[hive]( def this(sc: JavaSparkContext) = this(sc.sc) - 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 @@ -83,11 +81,6 @@ class HiveContext private[hive]( sparkSession.sharedState.asInstanceOf[HiveSharedState] } - protected[hive] def hiveCatalog: HiveExternalCatalog = sharedState.externalCatalog - protected[hive] def executionHive: HiveClientImpl = sessionState.executionHive - protected[hive] def metadataHive: HiveClient = sessionState.metadataHive - protected[hive] def hiveconf: HiveConf = sessionState.hiveconf - } 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 33cd8b44805b8..14dcd0cc44064 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 @@ -56,7 +56,7 @@ private[hive] case class CreateViewAsSelect( case true if orReplace => // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` - hiveContext.metadataHive.alertView(prepareTable(sqlContext)) + hiveContext.sessionState.metadataHive.alertView(prepareTable(sqlContext)) case true => // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already @@ -66,7 +66,7 @@ private[hive] case class CreateViewAsSelect( "CREATE OR REPLACE VIEW AS") case false => - hiveContext.metadataHive.createView(prepareTable(sqlContext)) + hiveContext.sessionState.metadataHive.createView(prepareTable(sqlContext)) } Seq.empty[Row] 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 ed538630d24a8..4f83c25e876d8 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 @@ -45,7 +45,7 @@ case class InsertIntoHiveTable( @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 lazy val client = sc.sessionState.metadataHive def output: Seq[Attribute] = Seq.empty diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 19c0077d2c848..2fb8cd1513160 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -60,7 +60,7 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { // Can we use fs.getContentSummary in future? // Seems fs.getContentSummary returns wrong table size on Jenkins. So we use // countFileSize to count the table size. - val stagingDir = hiveContext.metadataHive.getConf( + val stagingDir = hiveContext.sessionState.metadataHive.getConf( HiveConf.ConfVars.STAGINGDIR.varname, HiveConf.ConfVars.STAGINGDIR.defaultStrVal) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 13f0b8b0b4aca..34b610ea6c0da 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -424,7 +424,7 @@ class TestHiveContext private[hive]( loadedTables.clear() sessionState.catalog.clearTempTables() sessionState.catalog.invalidateCache() - metadataHive.reset() + sessionState.metadataHive.reset() FunctionRegistry.getFunctionNames.asScala.filterNot(originalUDFs.contains(_)). foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) } @@ -433,8 +433,8 @@ class TestHiveContext private[hive]( sessionState.hiveconf.set("fs.default.name", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. - executionHive.runSqlHive("RESET") - metadataHive.runSqlHive("RESET") + sessionState.executionHive.runSqlHive("RESET") + sessionState.metadataHive.runSqlHive("RESET") // For some reason, RESET does not reset the following variables... // https://issues.apache.org/jira/browse/HIVE-9004 sessionState.runNativeSql("set hive.table.parameters.default=") @@ -446,7 +446,7 @@ class TestHiveContext private[hive]( // In case a test changed any of these values, restore all the original ones here. TestHiveContext.hiveClientConfigurations( sessionState.hiveconf, warehousePath, scratchDirPath, metastoreTemporaryConf) - .foreach { case (k, v) => metadataHive.runSqlHive(s"SET $k=$v") } + .foreach { case (k, v) => sessionState.metadataHive.runSqlHive(s"SET $k=$v") } sessionState.setDefaultOverrideConfs() sessionState.catalog.setCurrentDatabase("default") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala index b644a50613337..ccc6dc57e9296 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala @@ -30,7 +30,8 @@ class HiveContextSuite extends SparkFunSuite { "org.apache.spark.sql.hive.execution.PairSerDe") assert(TestHive.initialSQLConf.getConfString("spark.sql.hive.metastore.barrierPrefixes") == "org.apache.spark.sql.hive.execution.PairSerDe") - assert(TestHive.metadataHive.getConf("spark.sql.hive.metastore.barrierPrefixes", "") == + assert(TestHive.sessionState.metadataHive.getConf( + "spark.sql.hive.metastore.barrierPrefixes", "") == "org.apache.spark.sql.hive.execution.PairSerDe") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 1b7f82cbbe9c5..c056d49afbf42 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -738,7 +738,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv "spark.sql.sources.schema" -> schema.json, "EXTERNAL" -> "FALSE")) - hiveCatalog.createTable("default", hiveTable, ignoreIfExists = false) + sharedState.externalCatalog.createTable("default", hiveTable, ignoreIfExists = false) sessionState.invalidateTable(tableName) val actualSchema = table(tableName).schema @@ -753,7 +753,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv withTable(tableName) { df.write.format("parquet").partitionBy("d", "b").saveAsTable(tableName) sessionState.invalidateTable(tableName) - val metastoreTable = hiveCatalog.getTable("default", tableName) + val metastoreTable = sharedState.externalCatalog.getTable("default", tableName) val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val numPartCols = metastoreTable.properties("spark.sql.sources.schema.numPartCols").toInt @@ -788,7 +788,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .sortBy("c") .saveAsTable(tableName) sessionState.invalidateTable(tableName) - val metastoreTable = hiveCatalog.getTable("default", tableName) + val metastoreTable = sharedState.externalCatalog.getTable("default", tableName) val expectedBucketByColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val expectedSortByColumns = StructType(df.schema("c") :: Nil) @@ -918,7 +918,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in Hive compatible format, // we verify that each column of the table is of native type StringType. - assert(hiveCatalog.getTable("default", "not_skip_hive_metadata").schema + assert(sharedState.externalCatalog.getTable("default", "not_skip_hive_metadata").schema .forall(column => HiveMetastoreTypes.toDataType(column.dataType) == StringType)) sessionState.catalog.createDataSourceTable( @@ -932,9 +932,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in SparkSQL format, // we verify that the table has a column type as array of StringType. - assert(hiveCatalog.getTable("default", "skip_hive_metadata").schema.forall { c => - HiveMetastoreTypes.toDataType(c.dataType) == ArrayType(StringType) - }) + assert(sharedState.externalCatalog.getTable("default", "skip_hive_metadata") + .schema.forall { c => HiveMetastoreTypes.toDataType(c.dataType) == ArrayType(StringType) }) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index e84109a27e2b6..850cb1eda5809 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -25,8 +25,9 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle private lazy val df = sqlContext.range(10).coalesce(1).toDF() private def checkTablePath(dbName: String, tableName: String): Unit = { - val metastoreTable = hiveContext.hiveCatalog.getTable(dbName, tableName) - val expectedPath = hiveContext.hiveCatalog.getDatabase(dbName).locationUri + "/" + tableName + val metastoreTable = hiveContext.sharedState.externalCatalog.getTable(dbName, tableName) + val expectedPath = + hiveContext.sharedState.externalCatalog.getDatabase(dbName).locationUri + "/" + tableName assert(metastoreTable.storage.serdeProperties("path") === expectedPath) } From edaebe5e5dbf03fe8425b1b4185c1a430b15bece Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 17:54:33 -0700 Subject: [PATCH 09/27] Fix style --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 1 - .../org/apache/spark/sql/hive/HiveSessionCatalog.scala | 2 +- .../scala/org/apache/spark/sql/hive/HiveSessionState.scala | 6 ++++-- .../scala/org/apache/spark/sql/hive/test/TestHive.scala | 3 ++- 4 files changed, 7 insertions(+), 5 deletions(-) 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 2ba1e3dc32afa..21bf202d67848 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 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 3adaca66bcb41..4f9513389c8c2 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, SQLContext} import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client.HiveClient 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 2706cc356b749..8ba0fc6ce7825 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 @@ -153,13 +153,15 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) // Helper methods, partially leftover from pre-2.0 days // ------------------------------------------------------ - override def executePlan(plan: LogicalPlan) = new HiveQueryExecution(ctx, plan) + 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 */ - private[sql] def setDefaultOverrideConfs(): Unit = { + def setDefaultOverrideConfs(): Unit = { setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false") } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 34b610ea6c0da..bb8531196211c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -245,7 +245,8 @@ class TestHiveContext private[hive]( } }), TestTable("srcpart1", () => { - sessionState.runNativeSql("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") + sessionState.runNativeSql( + "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { sessionState.runNativeSql( s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' From 4f3ade994b0e74fff018e17e918f721597c57e9e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 18:24:56 -0700 Subject: [PATCH 10/27] Minor fixes --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 14 ++++++-------- .../apache/spark/sql/hive/HiveQueryExecution.scala | 2 +- .../apache/spark/sql/hive/HiveSessionState.scala | 3 +-- 3 files changed, 8 insertions(+), 11 deletions(-) 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 72fd7ff617182..540d08b4d85d3 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 @@ -117,10 +117,10 @@ private[hive] object HiveSerDe { * cleaned up to integrate more nicely with [[HiveExternalCatalog]]. */ private[hive] class HiveMetastoreCatalog(hive: SQLContext) extends Logging { - val conf = hive.conf - val sessionState = hive.sessionState.asInstanceOf[HiveSessionState] - val client = hive.sharedState.asInstanceOf[HiveSharedState].metadataHive - val hiveconf = sessionState.hiveconf + 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) @@ -442,10 +442,8 @@ private[hive] class HiveMetastoreCatalog(hive: SQLContext) extends Logging { alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." - case None => - SubqueryAlias(table.identifier.table, hive.parseSql(viewText)) - case Some(aliasText) => - SubqueryAlias(aliasText, hive.parseSql(viewText)) + case None => SubqueryAlias(table.identifier.table, hive.parseSql(viewText)) + case Some(aliasText) => SubqueryAlias(aliasText, hive.parseSql(viewText)) } } else { MetastoreRelation( 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 index 66d2d84793416..1c1bfb610c29e 100644 --- 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNative /** * A [[QueryExecution]] with hive specific features. */ -protected[sql] class HiveQueryExecution(ctx: SQLContext, logicalPlan: LogicalPlan) +protected[hive] class HiveQueryExecution(ctx: SQLContext, logicalPlan: LogicalPlan) extends QueryExecution(ctx, logicalPlan) { /** 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 8ba0fc6ce7825..c15d2ca7486ae 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 @@ -74,7 +74,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) */ val hiveconf: HiveConf = { val c = executionHive.conf - conf.setConf(c.getAllProperties) + setConf(c.getAllProperties) c } @@ -186,7 +186,6 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * Right now, it only supports Hive tables and it only updates the size of a Hive table * in the Hive metastore. */ - // TODO: not used at the moment override def analyze(tableName: String): Unit = { AnalyzeTable(tableName).run(ctx) } From 60195413d268e5ff7f1c295cc6cbc60794406619 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 18:35:47 -0700 Subject: [PATCH 11/27] Use in-memory catalog by default in tests --- pom.xml | 2 ++ project/SparkBuild.scala | 1 + .../org/apache/spark/sql/SparkSession.scala | 28 +++++++++++++++++-- .../apache/spark/sql/hive/test/TestHive.scala | 3 +- 4 files changed, 30 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index a772d513372e7..8d80109077e07 100644 --- a/pom.xml +++ b/pom.xml @@ -1944,6 +1944,7 @@ false false false + in-memory true src @@ -1992,6 +1993,7 @@ 1 false false + in-memory true __not_used__ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a0df9b6a331f1..8a41154d36a97 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -718,6 +718,7 @@ object TestSettings { javaOptions in Test += "-Dspark.master.rest.enabled=false", javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", + javaOptions in Test += "-Dspark.sql.catalogImplementation=in-memory", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test += "-Dderby.system.durability=test", 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 index c97a5d5588391..b09a0b492d1fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.util.control.NonFatal -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.internal.{SessionState, SharedState} import org.apache.spark.util.Utils @@ -47,13 +47,13 @@ class SparkSession private( @transient protected[sql] lazy val sharedState: SharedState = { existingSharedState.getOrElse( - SparkSession.reflect(SparkSession.DEFAULT_SHARED_STATE_CLASS_NAME, sparkContext)) + SparkSession.reflect(SparkSession.sharedStateClassName(sparkContext.conf), sparkContext)) } @transient protected[sql] lazy val sessionState: SessionState = { SparkSession.reflect( - SparkSession.DEFAULT_SESSION_STATE_CLASS_NAME, + SparkSession.sessionStateClassName(sparkContext.conf), new SQLContext(self, isRootContext = false)) } @@ -65,6 +65,28 @@ private object SparkSession { private val DEFAULT_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" private val DEFAULT_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" + private def sharedStateClassName(conf: SparkConf): String = { + conf.getOption("spark.sql.catalogImplementation") match { + case Some("hive") => "org.apache.spark.sql.hive.HiveSharedState" + case Some("in-memory") => classOf[SharedState].getCanonicalName + case Some(unknown) => + throw new IllegalArgumentException( + s"Unexpected catalog implementation '$unknown'; must be 'hive' or 'in-memory'") + case None => DEFAULT_SHARED_STATE_CLASS_NAME + } + } + + private def sessionStateClassName(conf: SparkConf): String = { + conf.getOption("spark.sql.catalogImplementation") match { + case Some("hive") => "org.apache.spark.sql.hive.HiveSessionState" + case Some("in-memory") => classOf[SessionState].getCanonicalName + case Some(unknown) => + throw new IllegalArgumentException( + s"Unexpected catalog implementation '$unknown'; must be 'hive' or 'in-memory'") + case None => DEFAULT_SESSION_STATE_CLASS_NAME + } + } + /** * Helper method to create an instance of [[T]] using a single-arg constructor that * accepts an [[Arg]]. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index bb8531196211c..dc16a87276a34 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -55,7 +55,8 @@ object TestHive .set("spark.sql.hive.metastore.barrierPrefixes", "org.apache.spark.sql.hive.execution.PairSerDe") // SPARK-8910 - .set("spark.ui.enabled", "false"))) + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "hive"))) /** From 75d1115e140b8401343c0b96429ccd346e3a4b17 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 18:47:31 -0700 Subject: [PATCH 12/27] Fix NPE when initializing HiveSessionState --- .../org/apache/spark/sql/SparkSession.scala | 6 ++++-- .../spark/sql/hive/HiveSessionState.scala | 19 +++++++++---------- 2 files changed, 13 insertions(+), 12 deletions(-) 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 index b09a0b492d1fb..6ea0f49298130 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -47,12 +47,14 @@ class SparkSession private( @transient protected[sql] lazy val sharedState: SharedState = { existingSharedState.getOrElse( - SparkSession.reflect(SparkSession.sharedStateClassName(sparkContext.conf), sparkContext)) + SparkSession.reflect[SharedState, SparkContext]( + SparkSession.sharedStateClassName(sparkContext.conf), + sparkContext)) } @transient protected[sql] lazy val sessionState: SessionState = { - SparkSession.reflect( + SparkSession.reflect[SessionState, SQLContext]( SparkSession.sessionStateClassName(sparkContext.conf), new SQLContext(self, isRootContext = false)) } 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 c15d2ca7486ae..a0f340e9bd5bf 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 @@ -41,28 +41,27 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) self => - private val sharedState: HiveSharedState = ctx.sharedState.asInstanceOf[HiveSharedState] + private lazy val sharedState: HiveSharedState = ctx.sharedState.asInstanceOf[HiveSharedState] /** * A Hive client used for execution. */ - val executionHive: HiveClientImpl = sharedState.executionHive.newSession() + lazy val executionHive: HiveClientImpl = sharedState.executionHive.newSession() /** * A Hive client used for interacting with the metastore. */ - val metadataHive: HiveClient = sharedState.metadataHive.newSession() + lazy val metadataHive: HiveClient = sharedState.metadataHive.newSession() /** * A Hive helper class for substituting variables in a SQL statement. */ - val substitutor = new VariableSubstitution + lazy val substitutor = new VariableSubstitution override lazy val conf: SQLConf = new SQLConf { override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) } - setDefaultOverrideConfs() /** * SQLConf and HiveConf contracts: @@ -72,11 +71,11 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * 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. */ - val hiveconf: HiveConf = { - val c = executionHive.conf - setConf(c.getAllProperties) - c - } + lazy val hiveconf: HiveConf = executionHive.conf + + // Set some default confs + setDefaultOverrideConfs() + setConf(hiveconf.getAllProperties) /** * Internal catalog for managing table and database states. From 36d6bc891c2d591233e41e470815e0393718af7d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 18:56:45 -0700 Subject: [PATCH 13/27] Fix the conf --- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 8 ++++++-- .../org/apache/spark/sql/internal/SessionState.scala | 6 ++++++ 2 files changed, 12 insertions(+), 2 deletions(-) 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 d093e38f2c1fe..51681fe098425 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 @@ -136,8 +136,12 @@ class SQLContext private[sql]( */ 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. 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 dacf67c25e596..b86e4f5c4ef71 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 @@ -30,6 +30,7 @@ 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 +import org.apache.spark.internal.config.ConfigEntry /** @@ -130,6 +131,11 @@ private[sql] class SessionState(ctx: SQLContext) { 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) } From a1d45e8f819bead89f5b0b0f68a667d574b250a6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 11:06:45 -0700 Subject: [PATCH 14/27] Fix REPL in in-memory case Previously we still tried to load HiveContext even if the user explicitly specified an "in-memory" catalog impelmentation. Now it will load a SQLContext in this case. --- .../spark/internal/config/package.scala | 7 +++++ .../org/apache/spark/repl/SparkILoop.scala | 16 +---------- .../scala/org/apache/spark/repl/Main.scala | 27 ++++++++++++------- .../org/apache/spark/sql/SparkSession.scala | 24 +++++------------ 4 files changed, 32 insertions(+), 42 deletions(-) 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 94b50ee06520c..5cfd5c136b198 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("hive") } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index c5dc6ba2219f8..82d9708ce5a41 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1026,21 +1026,7 @@ class SparkILoop( } @DeveloperApi - def createSQLContext(): SQLContext = { - val name = "org.apache.spark.sql.hive.HiveContext" - val loader = Utils.getContextOrSparkClassLoader - try { - sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) - .newInstance(sparkContext).asInstanceOf[SQLContext] - logInfo("Created sql context (with Hive support)..") - } - catch { - case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => - sqlContext = new SQLContext(sparkContext) - logInfo("Created sql context..") - } - sqlContext - } + def createSQLContext(): SQLContext = Main.createSQLContext() private def getMaster(): String = { val master = this.master match { diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index b822ff496c118..943be0fc4f61e 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -23,6 +23,7 @@ import scala.tools.nsc.GenericRunnerSettings import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.util.Utils import org.apache.spark.sql.SQLContext @@ -92,16 +93,22 @@ object Main extends Logging { } def createSQLContext(): SQLContext = { - val name = "org.apache.spark.sql.hive.HiveContext" - val loader = Utils.getContextOrSparkClassLoader - try { - sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) - .newInstance(sparkContext).asInstanceOf[SQLContext] - logInfo("Created sql context (with Hive support)..") - } catch { - case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => - sqlContext = new SQLContext(sparkContext) - logInfo("Created sql context..") + val shouldTryHiveContext = sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive" + if (shouldTryHiveContext) { + val name = "org.apache.spark.sql.hive.HiveContext" + val loader = Utils.getContextOrSparkClassLoader + try { + sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) + .newInstance(sparkContext).asInstanceOf[SQLContext] + logInfo("Created sql context (with Hive support)..") + } catch { + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => + sqlContext = new SQLContext(sparkContext) + logInfo("Created sql context..") + } + } else { + sqlContext = new SQLContext(sparkContext) + logInfo("Created sql context..") } sqlContext } 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 index 6ea0f49298130..d5440798287ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql 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 @@ -64,28 +65,17 @@ class SparkSession private( private object SparkSession { - private val DEFAULT_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" - private val DEFAULT_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" - private def sharedStateClassName(conf: SparkConf): String = { - conf.getOption("spark.sql.catalogImplementation") match { - case Some("hive") => "org.apache.spark.sql.hive.HiveSharedState" - case Some("in-memory") => classOf[SharedState].getCanonicalName - case Some(unknown) => - throw new IllegalArgumentException( - s"Unexpected catalog implementation '$unknown'; must be 'hive' or 'in-memory'") - case None => DEFAULT_SHARED_STATE_CLASS_NAME + 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.getOption("spark.sql.catalogImplementation") match { - case Some("hive") => "org.apache.spark.sql.hive.HiveSessionState" - case Some("in-memory") => classOf[SessionState].getCanonicalName - case Some(unknown) => - throw new IllegalArgumentException( - s"Unexpected catalog implementation '$unknown'; must be 'hive' or 'in-memory'") - case None => DEFAULT_SESSION_STATE_CLASS_NAME + conf.get(CATALOG_IMPLEMENTATION) match { + case "hive" => "org.apache.spark.sql.hive.HiveSessionState" + case "in-memory" => classOf[SessionState].getCanonicalName } } From 0df39fc9082d2f9f89dc3712e1f49132713f2d38 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 11:20:47 -0700 Subject: [PATCH 15/27] Fix tests: set "in-memory" in more places --- .../test/scala/org/apache/spark/SharedSparkContext.scala | 3 ++- .../org/apache/spark/sql/MultiSQLContextsSuite.scala | 1 + .../spark/sql/execution/ExchangeCoordinatorSuite.scala | 1 + .../scala/org/apache/spark/sql/test/TestSQLContext.scala | 8 ++++++-- 4 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 858bc742e07cf..62343cbf0a646 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -27,7 +27,8 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => def sc: SparkContext = _sc - var conf = new SparkConf(false) + // TODO: why can't we just load defaults here? + var conf = new SparkConf(false).set("spark.sql.catalogImplementation", "in-memory") override def beforeAll() { super.beforeAll() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala index 0b5a92c256e57..7ef06b9bed248 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala @@ -40,6 +40,7 @@ class MultiSQLContextsSuite extends SparkFunSuite with BeforeAndAfterAll { .setAppName("test") .set("spark.ui.enabled", "false") .set("spark.driver.allowMultipleContexts", "true") + .set("spark.sql.catalogImplementation", "in-memory") } override protected def afterAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 01d485ce2d713..850a738428661 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -260,6 +260,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .setAppName("test") .set("spark.ui.enabled", "false") .set("spark.driver.allowMultipleContexts", "true") + .set("spark.sql.catalogImplementation", "in-memory") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala index 7ab79b12ce246..3461bad9d8093 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -27,8 +27,12 @@ import org.apache.spark.sql.internal.{SessionState, SQLConf} private[sql] class TestSQLContext(sc: SparkContext) extends SQLContext(sc) { self => def this(sparkConf: SparkConf) { - this(new SparkContext("local[2]", "test-sql-context", - sparkConf.set("spark.sql.testkey", "true"))) + this(new SparkContext( + "local[2]", + "test-sql-context", + sparkConf + .set("spark.sql.testkey", "true") + .set("spark.sql.catalogImplementation", "in-memory"))) } def this() { From 0d7309b20f72809659306aec47f29c8585f36dd7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 11:24:56 -0700 Subject: [PATCH 16/27] Fix style --- .../main/scala/org/apache/spark/sql/internal/SessionState.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b86e4f5c4ef71..42915d5887f44 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 @@ -21,6 +21,7 @@ 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 @@ -30,7 +31,6 @@ 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 -import org.apache.spark.internal.config.ConfigEntry /** From bc352068a40a13712e0f16bbc81d66486e4a3635 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 11:50:27 -0700 Subject: [PATCH 17/27] Fix SQLExecutionSuite It was failing because we were passing in a subclass of SparkContext into SparkSession, and the reflection was using the wrong class to get the constructor. This is now fixed with ClassTags. --- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 index d5440798287ca..17ba2998250f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkContext} @@ -83,10 +84,12 @@ private object SparkSession { * 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): T = { + 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(ctorArg.getClass) + val ctor = clazz.getDeclaredConstructor(ctorArgTag.runtimeClass) ctor.newInstance(ctorArg).asInstanceOf[T] } catch { case NonFatal(e) => From 5fcc249f4368963625e294ca3b1ea78b07b96252 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 12:09:43 -0700 Subject: [PATCH 18/27] Fix ParquetHadoopFsRelationSuite Avoid some unnecessary casts. --- .../hive/execution/CreateTableAsSelect.scala | 9 +++---- .../spark/sql/hive/execution/commands.scala | 26 +++++++++---------- .../apache/spark/sql/hive/test/TestHive.scala | 3 +++ 3 files changed, 19 insertions(+), 19 deletions(-) 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 29f7dc2997d26..ceb7f3b890949 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] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 0eb8d5648d774..6c34c4eee7ea5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSource, LogicalRelation} -import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} +import org.apache.spark.sql.hive.{HiveContext, HiveSessionState, MetastoreRelation} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -176,9 +176,9 @@ case class CreateMetastoreDataSource( } val tableName = tableIdent.unquotedString - val hiveContext = sqlContext.asInstanceOf[HiveContext] + val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] - if (hiveContext.sessionState.catalog.tableExists(tableIdent)) { + if (sessionState.catalog.tableExists(tableIdent)) { if (allowExisting) { return Seq.empty[Row] } else { @@ -190,8 +190,7 @@ case class CreateMetastoreDataSource( val optionsWithPath = if (!options.contains("path") && managedIfNoPath) { isExternal = false - options + ("path" -> - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) + options + ("path" -> sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } @@ -204,7 +203,7 @@ case class CreateMetastoreDataSource( bucketSpec = None, options = optionsWithPath).resolveRelation() - hiveContext.sessionState.catalog.createDataSourceTable( + sessionState.catalog.createDataSourceTable( tableIdent, userSpecifiedSchema, Array.empty[String], @@ -243,14 +242,13 @@ case class CreateMetastoreDataSourceAsSelect( } val tableName = tableIdent.unquotedString - val hiveContext = sqlContext.asInstanceOf[HiveContext] + val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] var createMetastoreTable = false var isExternal = true val optionsWithPath = if (!options.contains("path")) { isExternal = false - options + ("path" -> - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) + options + ("path" -> sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } @@ -281,14 +279,14 @@ case class CreateMetastoreDataSourceAsSelect( // inserting into (i.e. using the same compression). EliminateSubqueryAliases( - sqlContext.sessionState.catalog.lookupRelation(tableIdent)) match { + sessionState.catalog.lookupRelation(tableIdent)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => existingSchema = Some(l.schema) case o => throw new AnalysisException(s"Saving data in ${o.toString} is not supported.") } case SaveMode.Overwrite => - hiveContext.sql(s"DROP TABLE IF EXISTS $tableName") + sqlContext.sql(s"DROP TABLE IF EXISTS $tableName") // Need to create the table again. createMetastoreTable = true } @@ -297,7 +295,7 @@ case class CreateMetastoreDataSourceAsSelect( createMetastoreTable = true } - val data = Dataset.ofRows(hiveContext, query) + val data = Dataset.ofRows(sqlContext, query) val df = existingSchema match { // If we are inserting into an existing table, just use the existing schema. case Some(s) => data.selectExpr(s.fieldNames: _*) @@ -318,7 +316,7 @@ case class CreateMetastoreDataSourceAsSelect( // We will use the schema of resolved.relation as the schema of the table (instead of // the schema of df). It is important since the nullability may be changed by the relation // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). - hiveContext.sessionState.catalog.createDataSourceTable( + sessionState.catalog.createDataSourceTable( tableIdent, Some(result.schema), partitionColumns, @@ -329,7 +327,7 @@ case class CreateMetastoreDataSourceAsSelect( } // Refresh the cache of the table in the catalog. - hiveContext.sessionState.catalog.refreshTable(tableIdent) + sessionState.catalog.refreshTable(tableIdent) Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index dc16a87276a34..48d33e8b2b7a3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -470,6 +471,8 @@ private[hive] class TestHiveSparkSession( self => + assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") + override lazy val sharedState = new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf) From d9370385ed9d6f88a5aec46b71589cc94bc1f998 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 14:33:42 -0700 Subject: [PATCH 19/27] Fix HiveUDFSuite + refactor TestHive The problem was that we weren't using the right QueryExecution when we called TestHive.sessionState.executePlan. We were using HiveQueryExecution instead of the custom one that we created in TestHiveContext. This turned out to be very difficult to fix due to the tight coupling of QueryExecution within TestHiveContext. I had to refactor this code significantly to extract the nested logic one by one. --- .../execution/HiveCompatibilitySuite.scala | 4 +- .../HiveWindowFunctionQuerySuite.scala | 31 ++- .../apache/spark/sql/hive/test/TestHive.scala | 261 +++++++++--------- .../spark/sql/hive/HiveContextSuite.scala | 3 +- .../execution/BigDataBenchmarkSuite.scala | 4 +- .../hive/execution/HiveComparisonTest.scala | 21 +- .../sql/hive/execution/HiveQuerySuite.scala | 10 +- .../sql/hive/execution/HiveSerDeSuite.scala | 4 +- .../sql/hive/execution/PruningSuite.scala | 18 +- .../spark/sql/sources/BucketedReadSuite.scala | 2 +- 10 files changed, 189 insertions(+), 169 deletions(-) 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 11c3d722b00a4..49fd19873017d 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 @@ -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) 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 d0b4cbe401eb3..de592f8d937dd 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( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 48d33e8b2b7a3..67a502d4ce31a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ @@ -71,50 +72,77 @@ object TestHive * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * test cases that rely on TestHive must be serialized. */ -class TestHiveContext private[hive]( - sparkSession: SparkSession, - val warehousePath: File, - val scratchDirPath: File, - metastoreTemporaryConf: Map[String, String], - isRootContext: Boolean) - extends HiveContext(sparkSession, isRootContext) { self => +class TestHiveContext(val sparkSession: TestHiveSparkSession, isRootContext: Boolean) + extends HiveContext(sparkSession, isRootContext) { - private def this( - sc: SparkContext, - warehousePath: File, - scratchDirPath: File, - metastoreTemporaryConf: Map[String, String]) { - this( - new TestHiveSparkSession(sc, warehousePath, scratchDirPath, metastoreTemporaryConf), - warehousePath, - scratchDirPath, - metastoreTemporaryConf, - true) + def this(sc: SparkContext) { + this(new TestHiveSparkSession(sc), true) + } + + override def newSession(): TestHiveContext = { + new TestHiveContext(sparkSession.newSession(), false) + } + + override def sharedState: TestHiveSharedState = sparkSession.sharedState + + override def sessionState: TestHiveSessionState = sparkSession.sessionState + + def setCacheTables(c: Boolean): Unit = { + sparkSession.setCacheTables(c) + } + + def getHiveFile(path: String): File = { + sparkSession.getHiveFile(path) + } + + def loadTestTable(name: String): Unit = { + sparkSession.loadTestTable(name) } + def reset(): Unit = { + sparkSession.reset() + } + +} + + +private[hive] class TestHiveSparkSession( + sc: SparkContext, + val warehousePath: File, + scratchDirPath: File, + metastoreTemporaryConf: Map[String, String], + existingSharedState: Option[TestHiveSharedState]) + extends SparkSession(sc) with Logging { self => + def this(sc: SparkContext) { this( sc, Utils.createTempDir(namePrefix = "warehouse"), - TestHiveContext.makeScratchDir(), - HiveContext.newTemporaryConfiguration(useInMemoryDerby = false)) + TestHiveSparkSession.makeScratchDir(), + HiveContext.newTemporaryConfiguration(useInMemoryDerby = false), + None) } - override def newSession(): HiveContext = { - new TestHiveContext( - sparkSession.newSession(), - warehousePath, - scratchDirPath, - metastoreTemporaryConf, - isRootContext = false) + assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") + + @transient + override lazy val sharedState: TestHiveSharedState = { + existingSharedState.getOrElse( + new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf)) } - protected[sql] override def sessionState: TestHiveSessionState = { - sparkSession.sessionState.asInstanceOf[TestHiveSessionState] + @transient + override lazy val sessionState: TestHiveSessionState = new TestHiveSessionState(self) + + override def newSession(): TestHiveSparkSession = { + new TestHiveSparkSession( + sc, warehousePath, scratchDirPath, metastoreTemporaryConf, Some(sharedState)) } - protected[sql] override def sharedState: TestHiveSharedState = { - sparkSession.sharedState.asInstanceOf[TestHiveSharedState] + private var cacheTables: Boolean = false + + def setCacheTables(c: Boolean): Unit = { + cacheTables = c } // By clearing the port we force Spark to pick a new one. This allows us to rerun tests @@ -126,9 +154,9 @@ class TestHiveContext private[hive]( // A snapshot of the entries in the starting SQLConf // We save this because tests can mutate this singleton object if they want - val initialSQLConf: SQLConf = { + lazy val initialSQLConf: SQLConf = { val snapshot = new SQLConf - conf.getAllConfs.foreach { case (k, v) => snapshot.setConfString(k, v) } + sessionState.conf.getAllConfs.foreach { case (k, v) => snapshot.setConfString(k, v) } snapshot } @@ -141,11 +169,7 @@ class TestHiveContext private[hive]( lazy val hiveHome = envVarToFile("HIVE_HOME") /** The location of the hive source code. */ - lazy val hiveDevHome = { - val f = envVarToFile("HIVE_DEV_HOME") - sessionState.setHiveDevHome(f) - f - } + lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking @@ -160,16 +184,13 @@ class TestHiveContext private[hive]( hiveFilesTemp.mkdir() ShutdownHookManager.registerShutdownDeleteDir(hiveFilesTemp) - val inRepoTests: File = { - val f = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { + val inRepoTests: File = + if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) } else { new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + File.separator + "resources") } - sessionState.setInRepoTests(f) - f - } def getHiveFile(path: String): File = { val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar) @@ -181,36 +202,13 @@ class TestHiveContext private[hive]( val describedTable = "DESCRIBE (\\w+)".r - /** - * Override QueryExecution with special debug workflow. - */ - class QueryExecution(logicalPlan: LogicalPlan) - extends HiveQueryExecution(self, logicalPlan) { - def this(sql: String) = this(parseSql(sql)) - override lazy val analyzed = { - val describedTables = logical match { - case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil - case CacheTableCommand(tbl, _, _) => tbl :: Nil - case _ => Nil - } - - // Make sure any test tables referenced are loaded. - val referencedTables = - describedTables ++ - logical.collect { case UnresolvedRelation(tableIdent, _) => tableIdent.table } - val referencedTestTables = referencedTables.filter(testTables.contains) - logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") - referencedTestTables.foreach(loadTestTable) - // Proceed with analysis. - sessionState.analyzer.execute(logical) - } - } - case class TestTable(name: String, commands: (() => Unit)*) protected[hive] implicit class SqlCmd(sql: String) { def cmd: () => Unit = { - () => new QueryExecution(sql).stringResult(): Unit + // TODO: There's something wrong with CREATE TABLE followed by LOAD DATA INTO that table. + // See if we can replace this with `new TestHiveQueryExecution(self, sql).stringResult()` + () => sessionState.runNativeSql(sql) } } @@ -387,7 +385,6 @@ class TestHiveContext private[hive]( private val loadedTables = new collection.mutable.HashSet[String] - var cacheTables: Boolean = false def loadTestTable(name: String) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. @@ -398,11 +395,25 @@ class TestHiveContext private[hive]( createCmds.foreach(_()) if (cacheTables) { - cacheTable(name) + new SQLContext(self).cacheTable(name) } } } + /** + * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the + * hive test cases assume the system is set up. + */ + private[hive] def rewritePaths(cmd: String): String = { + if (cmd.toUpperCase contains "LOAD DATA") { + val testDataLocation = + hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) + cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") + } else { + cmd + } + } + /** * Records the UDFs present when the server starts, so we can delete ones that are created by * tests. @@ -423,7 +434,7 @@ class TestHiveContext private[hive]( } } - cacheManager.clearCache() + sharedState.cacheManager.clearCache() loadedTables.clear() sessionState.catalog.clearTempTables() sessionState.catalog.invalidateCache() @@ -447,7 +458,7 @@ class TestHiveContext private[hive]( sessionState.runNativeSql("set hive.metastore.partition.name.whitelist.pattern=.*") // In case a test changed any of these values, restore all the original ones here. - TestHiveContext.hiveClientConfigurations( + TestHiveSparkSession.hiveClientConfigurations( sessionState.hiveconf, warehousePath, scratchDirPath, metastoreTemporaryConf) .foreach { case (k, v) => sessionState.metadataHive.runSqlHive(s"SET $k=$v") } sessionState.setDefaultOverrideConfs() @@ -462,38 +473,35 @@ class TestHiveContext private[hive]( } -private[hive] class TestHiveSparkSession( - sc: SparkContext, - warehousePath: File, - scratchDirPath: File, - metastoreTemporaryConf: Map[String, String]) - extends SparkSession(sc) { - - self => - - assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") - - override lazy val sharedState = - new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf) - - override lazy val sessionState = - new TestHiveSessionState(new SQLContext(self)) -} - - -private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { +private[hive] class TestHiveQueryExecution( + sparkSession: TestHiveSparkSession, + logicalPlan: LogicalPlan) + extends HiveQueryExecution(new SQLContext(sparkSession), logicalPlan) with Logging { - private val removedFunctions = - collection.mutable.ArrayBuffer.empty[(String, (ExpressionInfo, FunctionBuilder))] + def this(sparkSession: TestHiveSparkSession, sql: String) { + this(sparkSession, sparkSession.sessionState.sqlParser.parsePlan(sql)) + } - def unregisterFunction(name: String): Unit = { - functionBuilders.remove(name).foreach(f => removedFunctions += name -> f) + def this(sql: String) { + this(TestHive.sparkSession, sql) } - def restore(): Unit = { - removedFunctions.foreach { - case (name, (info, builder)) => registerFunction(name, info, builder) + override lazy val analyzed = { + val describedTables = logical match { + case HiveNativeCommand(sparkSession.describedTable(tbl)) => tbl :: Nil + case CacheTableCommand(tbl, _, _) => tbl :: Nil + case _ => Nil } + + // Make sure any test tables referenced are loaded. + val referencedTables = + describedTables ++ + logical.collect { case UnresolvedRelation(tableIdent, _) => tableIdent.table } + val referencedTestTables = referencedTables.filter(sparkSession.testTables.contains) + logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + referencedTestTables.foreach(sparkSession.loadTestTable) + // Proceed with analysis. + sparkSession.sessionState.analyzer.execute(logical) } } @@ -506,23 +514,14 @@ private[hive] class TestHiveSharedState( extends HiveSharedState(sc) { override lazy val metadataHive: HiveClient = { - TestHiveContext.newClientForMetadata( + TestHiveSparkSession.newClientForMetadata( sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath, metastoreTemporaryConf) } - } -protected[hive] class TestHiveSessionState(sqlContext: SQLContext) - extends HiveSessionState(sqlContext) { - - // Hack alert: These will be set in TestHiveContext constructor. Due to initialization order - // constraints we can't pass them in through the session state constructor. Sorry to build on - // top of this mess! - private var hiveDevHome: Option[File] = None - private var inRepoTests: File = _ - def setHiveDevHome(f: Option[File]): Unit = { hiveDevHome = f } - def setInRepoTests(f: File): Unit = { inRepoTests = f } +private[hive] class TestHiveSessionState(sparkSession: TestHiveSparkSession) + extends HiveSessionState(new SQLContext(sparkSession)) { override lazy val conf: SQLConf = { new SQLConf { @@ -530,14 +529,14 @@ protected[hive] class TestHiveSessionState(sqlContext: SQLContext) override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) override def clear(): Unit = { super.clear() - TestHiveContext.overrideConfs.map { + TestHiveSparkSession.overrideConfs.map { case (key, value) => setConfString(key, value) } } } } - override lazy val functionRegistry = { + override lazy val functionRegistry: TestHiveFunctionRegistry = { // We use TestHiveFunctionRegistry at here to track functions that have been explicitly // unregistered (through TestHiveFunctionRegistry.unregisterFunction method). val fr = new TestHiveFunctionRegistry @@ -547,29 +546,35 @@ protected[hive] class TestHiveSessionState(sqlContext: SQLContext) fr } + override def executePlan(plan: LogicalPlan): TestHiveQueryExecution = { + new TestHiveQueryExecution(sparkSession, plan) + } + // Override so we can intercept relative paths and rewrite them to point at hive. override def runNativeSql(sql: String): Seq[String] = { - super.runNativeSql(rewritePaths(substitutor.substitute(hiveconf, sql))) + super.runNativeSql(sparkSession.rewritePaths(substitutor.substitute(hiveconf, sql))) } +} - /** - * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the - * hive test cases assume the system is set up. - */ - private def rewritePaths(cmd: String): String = { - if (cmd.toUpperCase contains "LOAD DATA") { - val testDataLocation = - hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) - cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") - } else { - cmd - } + +private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { + + private val removedFunctions = + collection.mutable.ArrayBuffer.empty[(String, (ExpressionInfo, FunctionBuilder))] + + def unregisterFunction(name: String): Unit = { + functionBuilders.remove(name).foreach(f => removedFunctions += name -> f) } + def restore(): Unit = { + removedFunctions.foreach { + case (name, (info, builder)) => registerFunction(name, info, builder) + } + } } -private[hive] object TestHiveContext { +private[hive] object TestHiveSparkSession { /** * A map used to store all confs that need to be overridden in sql/hive unit tests. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala index ccc6dc57e9296..a1d8945074039 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala @@ -28,7 +28,8 @@ class HiveContextSuite extends SparkFunSuite { val sc = TestHive.sparkContext require(sc.conf.get("spark.sql.hive.metastore.barrierPrefixes") == "org.apache.spark.sql.hive.execution.PairSerDe") - assert(TestHive.initialSQLConf.getConfString("spark.sql.hive.metastore.barrierPrefixes") == + assert(TestHive.sparkSession.initialSQLConf.getConfString( + "spark.sql.hive.metastore.barrierPrefixes") == "org.apache.spark.sql.hive.execution.PairSerDe") assert(TestHive.sessionState.metadataHive.getConf( "spark.sql.hive.metastore.barrierPrefixes", "") == diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index a3f5921a0cb23..c58a66418991b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -19,15 +19,15 @@ package org.apache.spark.sql.hive.execution import java.io.File -import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") + import org.apache.spark.sql.hive.test.TestHive.sparkSession._ + val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") val userVisitPath = new File(testDataDirectory, "uservisits").getCanonicalPath val testTables = Seq( TestTable( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 384577f642b79..bd46cb922e1ba 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -30,8 +30,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.command.{ExplainCommand, SetCommand} import org.apache.spark.sql.execution.datasources.DescribeCommand -import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable, SQLBuilder} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable} +import org.apache.spark.sql.hive.SQLBuilder +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} /** * Allows the creations of tests that execute the same query against both hive @@ -141,7 +142,7 @@ abstract class HiveComparisonTest } protected def prepareAnswer( - hiveQuery: TestHive.type#QueryExecution, + hiveQuery: TestHiveQueryExecution, answer: Seq[String]): Seq[String] = { def isSorted(plan: LogicalPlan): Boolean = plan match { @@ -332,7 +333,7 @@ abstract class HiveComparisonTest hiveCachedResults } else { - val hiveQueries = queryList.map(new TestHive.QueryExecution(_)) + val hiveQueries = queryList.map(new TestHiveQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. // Note this must only look at the logical plan as we might not be able to analyze if // other DDL has not been executed yet. @@ -382,10 +383,10 @@ abstract class HiveComparisonTest // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - var query: TestHive.QueryExecution = null + var query: TestHiveQueryExecution = null try { query = { - val originalQuery = new TestHive.QueryExecution(queryString) + val originalQuery = new TestHiveQueryExecution(queryString) val containsCommands = originalQuery.analyzed.collectFirst { case _: Command => () case _: LogicalInsertIntoHiveTable => () @@ -409,7 +410,7 @@ abstract class HiveComparisonTest } try { - val queryExecution = new TestHive.QueryExecution(convertedSQL) + val queryExecution = new TestHiveQueryExecution(convertedSQL) // Trigger the analysis of this converted SQL query. queryExecution.analyzed queryExecution @@ -472,12 +473,12 @@ abstract class HiveComparisonTest // If this query is reading other tables that were created during this test run // also print out the query plans and results for those. val computedTablesMessages: String = try { - val tablesRead = new TestHive.QueryExecution(query).executedPlan.collect { + val tablesRead = new TestHiveQueryExecution(query).executedPlan.collect { case ts: HiveTableScan => ts.relation.tableName }.toSet TestHive.reset() - val executions = queryList.map(new TestHive.QueryExecution(_)) + val executions = queryList.map(new TestHiveQueryExecution(_)) executions.foreach(_.toRdd) val tablesGenerated = queryList.zip(executions).flatMap { // We should take executedPlan instead of sparkPlan, because in following codes we @@ -562,7 +563,7 @@ abstract class HiveComparisonTest // okay by running a simple query. If this fails then we halt testing since // something must have gone seriously wrong. try { - new TestHive.QueryExecution("SELECT key FROM src").stringResult() + new TestHiveQueryExecution("SELECT key FROM src").stringResult() TestHive.sessionState.runNativeSql("SELECT key FROM src") } catch { case e: Exception => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2e7a1d921b75c..5ee021664d1c9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoin import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} +import org.apache.spark.sql.hive.test.{TestHive, TestHiveSparkSession} import org.apache.spark.sql.hive.test.TestHive._ case class TestData(a: Int, b: String) @@ -49,7 +49,7 @@ class HiveQuerySuite extends HiveComparisonTest 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 @@ -58,7 +58,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { override def afterAll() { try { - TestHive.cacheTables = false + TestHive.setCacheTables(false) TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") @@ -1009,7 +1009,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .mkString("/") // Loads partition data to a temporary table to verify contents - val path = s"$warehousePath/dynamic_part_table/$partFolder/part-00000" + val path = s"${sparkSession.warehousePath}/dynamic_part_table/$partFolder/part-00000" sql("DROP TABLE IF EXISTS dp_verify") sql("CREATE TABLE dp_verify(intcol INT)") @@ -1150,7 +1150,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { // "SET" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... - assert(sql("SET").collect().size === TestHiveContext.overrideConfs.size) + assert(sql("SET").collect().size === TestHiveSparkSession.overrideConfs.size) val defaults = collectResults(sql("SET")) assertResult(Set(testKey -> testVal)) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 5586a793618bd..b8af0b39c8392 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -28,8 +28,8 @@ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { override def beforeAll(): Unit = { import TestHive._ import org.apache.hadoop.hive.serde2.RegexSerDe - super.beforeAll() - TestHive.cacheTables = false + super.beforeAll() + TestHive.setCacheTables(false) sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 97cb9d972081c..79ac53c8630f9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -21,18 +21,22 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} /** * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { - TestHive.cacheTables = false - // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset - // the environment to ensure all referenced tables in this suites are not cached in-memory. - // Refer to https://issues.apache.org/jira/browse/SPARK-2283 for details. - TestHive.reset() + override def beforeAll(): Unit = { + super.beforeAll() + TestHive.setCacheTables(false) + // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, + // need to reset the environment to ensure all referenced tables in this suites are + // not cached in-memory. Refer to https://issues.apache.org/jira/browse/SPARK-2283 + // for details. + TestHive.reset() + } // Column pruning tests @@ -144,7 +148,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]): Unit = { test(s"$testCaseName - pruning test") { - val plan = new TestHive.QueryExecution(sql).sparkPlan + val plan = new TestHiveQueryExecution(sql).sparkPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index a0be55cfba94c..aa6101f7b73cf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -349,7 +349,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") - val tableDir = new File(hiveContext.warehousePath, "bucketed_table") + val tableDir = new File(hiveContext.sparkSession.warehousePath, "bucketed_table") Utils.deleteRecursively(tableDir) df1.write.parquet(tableDir.getAbsolutePath) From 303f991576f0a082b2d8cb4d98e9a866bb7d07a1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 14:46:02 -0700 Subject: [PATCH 20/27] Make diff slightly smaller? --- .../apache/spark/sql/hive/test/TestHive.scala | 90 +++++++++---------- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 2 files changed, 45 insertions(+), 47 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 67a502d4ce31a..fe98da4524d4e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -118,7 +118,7 @@ private[hive] class TestHiveSparkSession( this( sc, Utils.createTempDir(namePrefix = "warehouse"), - TestHiveSparkSession.makeScratchDir(), + TestHiveContext.makeScratchDir(), HiveContext.newTemporaryConfiguration(useInMemoryDerby = false), None) } @@ -179,18 +179,30 @@ private[hive] class TestHiveSparkSession( Option(System.getenv(envVar)).map(new File(_)) } + /** + * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the + * hive test cases assume the system is set up. + */ + private[hive] def rewritePaths(cmd: String): String = + if (cmd.toUpperCase contains "LOAD DATA") { + val testDataLocation = + hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) + cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") + } else { + cmd + } + val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() ShutdownHookManager.registerShutdownDeleteDir(hiveFilesTemp) - val inRepoTests: File = - if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { - new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) - } else { - new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + - File.separator + "resources") - } + val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { + new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) + } else { + new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + + File.separator + "resources") + } def getHiveFile(path: String): File = { val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar) @@ -400,20 +412,6 @@ private[hive] class TestHiveSparkSession( } } - /** - * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the - * hive test cases assume the system is set up. - */ - private[hive] def rewritePaths(cmd: String): String = { - if (cmd.toUpperCase contains "LOAD DATA") { - val testDataLocation = - hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) - cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") - } else { - cmd - } - } - /** * Records the UDFs present when the server starts, so we can delete ones that are created by * tests. @@ -458,7 +456,7 @@ private[hive] class TestHiveSparkSession( sessionState.runNativeSql("set hive.metastore.partition.name.whitelist.pattern=.*") // In case a test changed any of these values, restore all the original ones here. - TestHiveSparkSession.hiveClientConfigurations( + TestHiveContext.hiveClientConfigurations( sessionState.hiveconf, warehousePath, scratchDirPath, metastoreTemporaryConf) .foreach { case (k, v) => sessionState.metadataHive.runSqlHive(s"SET $k=$v") } sessionState.setDefaultOverrideConfs() @@ -486,7 +484,7 @@ private[hive] class TestHiveQueryExecution( this(TestHive.sparkSession, sql) } - override lazy val analyzed = { + override lazy val analyzed: LogicalPlan = { val describedTables = logical match { case HiveNativeCommand(sparkSession.describedTable(tbl)) => tbl :: Nil case CacheTableCommand(tbl, _, _) => tbl :: Nil @@ -506,6 +504,23 @@ private[hive] class TestHiveQueryExecution( } +private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { + + private val removedFunctions = + collection.mutable.ArrayBuffer.empty[(String, (ExpressionInfo, FunctionBuilder))] + + def unregisterFunction(name: String): Unit = { + functionBuilders.remove(name).foreach(f => removedFunctions += name -> f) + } + + def restore(): Unit = { + removedFunctions.foreach { + case (name, (info, builder)) => registerFunction(name, info, builder) + } + } +} + + private[hive] class TestHiveSharedState( sc: SparkContext, warehousePath: File, @@ -514,7 +529,7 @@ private[hive] class TestHiveSharedState( extends HiveSharedState(sc) { override lazy val metadataHive: HiveClient = { - TestHiveSparkSession.newClientForMetadata( + TestHiveContext.newClientForMetadata( sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath, metastoreTemporaryConf) } } @@ -529,7 +544,7 @@ private[hive] class TestHiveSessionState(sparkSession: TestHiveSparkSession) override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) override def clear(): Unit = { super.clear() - TestHiveSparkSession.overrideConfs.map { + TestHiveContext.overrideConfs.map { case (key, value) => setConfString(key, value) } } @@ -557,24 +572,7 @@ private[hive] class TestHiveSessionState(sparkSession: TestHiveSparkSession) } -private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { - - private val removedFunctions = - collection.mutable.ArrayBuffer.empty[(String, (ExpressionInfo, FunctionBuilder))] - - def unregisterFunction(name: String): Unit = { - functionBuilders.remove(name).foreach(f => removedFunctions += name -> f) - } - - def restore(): Unit = { - removedFunctions.foreach { - case (name, (info, builder)) => registerFunction(name, info, builder) - } - } -} - - -private[hive] object TestHiveSparkSession { +private[hive] object TestHiveContext { /** * A map used to store all confs that need to be overridden in sql/hive unit tests. @@ -605,7 +603,7 @@ private[hive] object TestHiveSparkSession { /** * Configurations needed to create a [[HiveClient]]. */ - private def hiveClientConfigurations( + def hiveClientConfigurations( hiveconf: HiveConf, warehousePath: File, scratchDirPath: File, @@ -617,7 +615,7 @@ private[hive] object TestHiveSparkSession { ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY.varname -> "1") } - private def makeScratchDir(): File = { + def makeScratchDir(): File = { val scratchDir = Utils.createTempDir(namePrefix = "scratch") scratchDir.delete() scratchDir diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 5ee021664d1c9..3e8f371ac9e64 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -1150,7 +1150,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { // "SET" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... - assert(sql("SET").collect().size === TestHiveSparkSession.overrideConfs.size) + assert(sql("SET").collect().size === TestHiveContext.overrideConfs.size) val defaults = collectResults(sql("SET")) assertResult(Set(testKey -> testVal)) { From d27ec500c11e0e7d69e2ac0c7bd9c80901944a63 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 14:50:22 -0700 Subject: [PATCH 21/27] Fix test compile --- .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 3e8f371ac9e64..93d63f2241324 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoin import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.test.{TestHive, TestHiveSparkSession} +import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} import org.apache.spark.sql.hive.test.TestHive._ case class TestData(a: Int, b: String) From ddc752a134cdea5b06386b0d4a3e7436dc5ba777 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 14:59:47 -0700 Subject: [PATCH 22/27] Fix HiveResolutionSuite --- .../hive/execution/InsertIntoHiveTable.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) 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 4f83c25e876d8..e614daadf3918 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.sessionState.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 From 9b8dc3a6efb278b8dceee2f1d491857d79d185db Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 15:09:41 -0700 Subject: [PATCH 23/27] Fix StatisticsSuite --- .../sql/hive/execution/CreateViewAsSelect.scala | 11 +++++------ .../apache/spark/sql/hive/execution/commands.scala | 12 +++++------- 2 files changed, 10 insertions(+), 13 deletions(-) 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 14dcd0cc44064..1e234d8508b40 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.sessionState.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.sessionState.metadataHive.createView(prepareTable(sqlContext)) + sessionState.metadataHive.createView(prepareTable(sqlContext)) } Seq.empty[Row] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 6c34c4eee7ea5..ed33029b1309c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -45,8 +45,7 @@ private[hive] case class AnalyzeTable(tableName: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { - val sessionState = sqlContext.sessionState - val hiveContext = sqlContext.asInstanceOf[HiveContext] + val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) @@ -60,7 +59,7 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { // Can we use fs.getContentSummary in future? // Seems fs.getContentSummary returns wrong table size on Jenkins. So we use // countFileSize to count the table size. - val stagingDir = hiveContext.sessionState.metadataHive.getConf( + val stagingDir = sessionState.metadataHive.getConf( HiveConf.ConfVars.STAGINGDIR.varname, HiveConf.ConfVars.STAGINGDIR.defaultStrVal) @@ -106,7 +105,7 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { .map(_.toLong) .getOrElse(0L) val newTotalSize = - getFileSizeForTable(hiveContext.sessionState.hiveconf, relation.hiveQlTable) + getFileSizeForTable(sessionState.hiveconf, relation.hiveQlTable) // Update the Hive metastore if the total size of the table is different than the size // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). @@ -144,9 +143,8 @@ private[hive] case class AddFile(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] - hiveContext.sessionState.runNativeSql(s"ADD FILE $path") - hiveContext.sparkContext.addFile(path) + sqlContext.sessionState.runNativeSql(s"ADD FILE $path") + sqlContext.sparkContext.addFile(path) Seq.empty[Row] } } From e2571372c80a3ef950c16b7a3669affc4154b8ad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 15:49:33 -0700 Subject: [PATCH 24/27] Minor change --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 51681fe098425..06f5049c914f1 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 @@ -761,7 +761,7 @@ class SQLContext private[sql]( * @since 1.3.0 */ def sql(sqlText: String): DataFrame = { - Dataset.ofRows(this, sessionState.sqlParser.parsePlan(sqlText)) + Dataset.ofRows(this, parseSql(sqlText)) } /** From 8bf123601db1d33455f82d30aa19bdd875bd00d7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 19 Apr 2016 18:06:39 -0700 Subject: [PATCH 25/27] Fix HiveUDFSuite (and many others) The problem was that we were getting everything from executionHive's hiveconf and setting that in metadataHive, overriding the value of `hive.metastore.warehouse.dir`, which we customize in TestHive. This resulted in a bunch of "Table src does not exist" errors from Hive. --- .../org/apache/spark/sql/hive/HiveSessionState.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 a0f340e9bd5bf..bd48860ebe443 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 @@ -71,11 +71,13 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * 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 = executionHive.conf + lazy val hiveconf: HiveConf = { + val c = executionHive.conf + conf.setConf(c.getAllProperties) + c + } - // Set some default confs setDefaultOverrideConfs() - setConf(hiveconf.getAllProperties) /** * Internal catalog for managing table and database states. From 32212bbe6c3dc886470215c7be5eae237cfe008c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 20 Apr 2016 00:59:34 -0700 Subject: [PATCH 26/27] Fix tests --- .../main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- .../org/apache/spark/sql/hive/HiveSessionState.scala | 2 +- .../apache/spark/sql/hive/client/HiveClientImpl.scala | 2 ++ .../org/apache/spark/sql/hive/test/TestHive.scala | 10 +++++----- .../org/apache/spark/sql/hive/HiveContextSuite.scala | 3 ++- 5 files changed, 11 insertions(+), 8 deletions(-) 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 06f5049c914f1..f3f84144ad93e 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 @@ -204,7 +204,7 @@ class SQLContext private[sql]( * 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. */ 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 bd48860ebe443..09297c27dc5bc 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 @@ -85,7 +85,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) override lazy val catalog = { new HiveSessionCatalog( sharedState.externalCatalog, - sharedState.metadataHive, + metadataHive, ctx, ctx.functionResourceLoader, functionRegistry, 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 2a1fff92b570a..69f7dbf6ceebf 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 = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index fe98da4524d4e..504609d4acb34 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -72,7 +72,7 @@ object TestHive * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * test cases that rely on TestHive must be serialized. */ -class TestHiveContext(val sparkSession: TestHiveSparkSession, isRootContext: Boolean) +class TestHiveContext(@transient val sparkSession: TestHiveSparkSession, isRootContext: Boolean) extends HiveContext(sparkSession, isRootContext) { def this(sc: SparkContext) { @@ -154,7 +154,8 @@ private[hive] class TestHiveSparkSession( // A snapshot of the entries in the starting SQLConf // We save this because tests can mutate this singleton object if they want - lazy val initialSQLConf: SQLConf = { + // This snapshot is saved when we create this TestHiveSparkSession. + val initialSQLConf: SQLConf = { val snapshot = new SQLConf sessionState.conf.getAllConfs.foreach { case (k, v) => snapshot.setConfString(k, v) } snapshot @@ -218,9 +219,7 @@ private[hive] class TestHiveSparkSession( protected[hive] implicit class SqlCmd(sql: String) { def cmd: () => Unit = { - // TODO: There's something wrong with CREATE TABLE followed by LOAD DATA INTO that table. - // See if we can replace this with `new TestHiveQueryExecution(self, sql).stringResult()` - () => sessionState.runNativeSql(sql) + () => new TestHiveQueryExecution(sql).stringResult(): Unit } } @@ -436,6 +435,7 @@ private[hive] class TestHiveSparkSession( loadedTables.clear() sessionState.catalog.clearTempTables() sessionState.catalog.invalidateCache() + sessionState.metadataHive.reset() FunctionRegistry.getFunctionNames.asScala.filterNot(originalUDFs.contains(_)). diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala index a1d8945074039..b2c0f7e0e57b4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala @@ -31,7 +31,8 @@ class HiveContextSuite extends SparkFunSuite { assert(TestHive.sparkSession.initialSQLConf.getConfString( "spark.sql.hive.metastore.barrierPrefixes") == "org.apache.spark.sql.hive.execution.PairSerDe") - assert(TestHive.sessionState.metadataHive.getConf( + // This setting should be also set in the hiveconf of the current session. + assert(TestHive.sessionState.hiveconf.get( "spark.sql.hive.metastore.barrierPrefixes", "") == "org.apache.spark.sql.hive.execution.PairSerDe") } From 6e3c366a71f2dff1bbff59fd662b334fcc4798b8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 20 Apr 2016 09:35:35 -0700 Subject: [PATCH 27/27] Fix Python tests. It may take time to track all places where we only use SQLContext. So, let's change the catalog conf's default value to in-memory. In the constructor of HiveContext, we will set this conf to hive. --- .../spark/internal/config/package.scala | 2 +- .../org/apache/spark/SharedSparkContext.scala | 3 +-- pom.xml | 2 -- project/SparkBuild.scala | 1 - .../org/apache/spark/repl/SparkILoop.scala | 16 ++++++++++- .../scala/org/apache/spark/repl/Main.scala | 27 +++++++------------ .../spark/sql/MultiSQLContextsSuite.scala | 1 - .../execution/ExchangeCoordinatorSuite.scala | 1 - .../spark/sql/test/TestSQLContext.scala | 8 ++---- .../apache/spark/sql/hive/HiveContext.scala | 9 ++++++- .../apache/spark/sql/hive/test/TestHive.scala | 8 +++--- 11 files changed, 42 insertions(+), 36 deletions(-) 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 5cfd5c136b198..2c1e0b71e3613 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 @@ -95,5 +95,5 @@ package object config { .internal() .stringConf .checkValues(Set("hive", "in-memory")) - .createWithDefault("hive") + .createWithDefault("in-memory") } diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 62343cbf0a646..858bc742e07cf 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -27,8 +27,7 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => def sc: SparkContext = _sc - // TODO: why can't we just load defaults here? - var conf = new SparkConf(false).set("spark.sql.catalogImplementation", "in-memory") + var conf = new SparkConf(false) override def beforeAll() { super.beforeAll() diff --git a/pom.xml b/pom.xml index 8d80109077e07..a772d513372e7 100644 --- a/pom.xml +++ b/pom.xml @@ -1944,7 +1944,6 @@ false false false - in-memory true src @@ -1993,7 +1992,6 @@ 1 false false - in-memory true __not_used__ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8a41154d36a97..a0df9b6a331f1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -718,7 +718,6 @@ object TestSettings { javaOptions in Test += "-Dspark.master.rest.enabled=false", javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", - javaOptions in Test += "-Dspark.sql.catalogImplementation=in-memory", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test += "-Dderby.system.durability=test", diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 82d9708ce5a41..c5dc6ba2219f8 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1026,7 +1026,21 @@ class SparkILoop( } @DeveloperApi - def createSQLContext(): SQLContext = Main.createSQLContext() + def createSQLContext(): SQLContext = { + val name = "org.apache.spark.sql.hive.HiveContext" + val loader = Utils.getContextOrSparkClassLoader + try { + sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) + .newInstance(sparkContext).asInstanceOf[SQLContext] + logInfo("Created sql context (with Hive support)..") + } + catch { + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => + sqlContext = new SQLContext(sparkContext) + logInfo("Created sql context..") + } + sqlContext + } private def getMaster(): String = { val master = this.master match { diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 943be0fc4f61e..b822ff496c118 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -23,7 +23,6 @@ import scala.tools.nsc.GenericRunnerSettings import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.util.Utils import org.apache.spark.sql.SQLContext @@ -93,22 +92,16 @@ object Main extends Logging { } def createSQLContext(): SQLContext = { - val shouldTryHiveContext = sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive" - if (shouldTryHiveContext) { - val name = "org.apache.spark.sql.hive.HiveContext" - val loader = Utils.getContextOrSparkClassLoader - try { - sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) - .newInstance(sparkContext).asInstanceOf[SQLContext] - logInfo("Created sql context (with Hive support)..") - } catch { - case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => - sqlContext = new SQLContext(sparkContext) - logInfo("Created sql context..") - } - } else { - sqlContext = new SQLContext(sparkContext) - logInfo("Created sql context..") + val name = "org.apache.spark.sql.hive.HiveContext" + val loader = Utils.getContextOrSparkClassLoader + try { + sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) + .newInstance(sparkContext).asInstanceOf[SQLContext] + logInfo("Created sql context (with Hive support)..") + } catch { + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => + sqlContext = new SQLContext(sparkContext) + logInfo("Created sql context..") } sqlContext } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala index 7ef06b9bed248..0b5a92c256e57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala @@ -40,7 +40,6 @@ class MultiSQLContextsSuite extends SparkFunSuite with BeforeAndAfterAll { .setAppName("test") .set("spark.ui.enabled", "false") .set("spark.driver.allowMultipleContexts", "true") - .set("spark.sql.catalogImplementation", "in-memory") } override protected def afterAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 850a738428661..01d485ce2d713 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -260,7 +260,6 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .setAppName("test") .set("spark.ui.enabled", "false") .set("spark.driver.allowMultipleContexts", "true") - .set("spark.sql.catalogImplementation", "in-memory") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala index 3461bad9d8093..7ab79b12ce246 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -27,12 +27,8 @@ import org.apache.spark.sql.internal.{SessionState, SQLConf} private[sql] class TestSQLContext(sc: SparkContext) extends SQLContext(sc) { self => def this(sparkConf: SparkConf) { - this(new SparkContext( - "local[2]", - "test-sql-context", - sparkConf - .set("spark.sql.testkey", "true") - .set("spark.sql.catalogImplementation", "in-memory"))) + this(new SparkContext("local[2]", "test-sql-context", + sparkConf.set("spark.sql.testkey", "true"))) } def this() { 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 21bf202d67848..b2ce3e0df25b4 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 @@ -37,6 +37,7 @@ 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.CATALOG_IMPLEMENTATION import org.apache.spark.sql._ import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.internal.SQLConf @@ -58,7 +59,7 @@ class HiveContext private[hive]( self => def this(sc: SparkContext) = { - this(new SparkSession(sc), true) + this(new SparkSession(HiveContext.withHiveExternalCatalog(sc)), true) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -84,6 +85,12 @@ class HiveContext private[hive]( private[hive] object HiveContext extends Logging { + + def withHiveExternalCatalog(sc: SparkContext): SparkContext = { + sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive") + sc + } + /** The version of hive used internally by Spark SQL. */ val hiveExecutionVersion: String = "1.2.1" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 504609d4acb34..e629099086899 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -57,8 +57,7 @@ object TestHive .set("spark.sql.hive.metastore.barrierPrefixes", "org.apache.spark.sql.hive.execution.PairSerDe") // SPARK-8910 - .set("spark.ui.enabled", "false") - .set("spark.sql.catalogImplementation", "hive"))) + .set("spark.ui.enabled", "false"))) /** @@ -76,7 +75,7 @@ class TestHiveContext(@transient val sparkSession: TestHiveSparkSession, isRootC extends HiveContext(sparkSession, isRootContext) { def this(sc: SparkContext) { - this(new TestHiveSparkSession(sc), true) + this(new TestHiveSparkSession(HiveContext.withHiveExternalCatalog(sc)), true) } override def newSession(): TestHiveContext = { @@ -125,6 +124,9 @@ private[hive] class TestHiveSparkSession( assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") + // TODO: Let's remove TestHiveSharedState and TestHiveSessionState. Otherwise, + // we are not really testing the reflection logic based on the setting of + // CATALOG_IMPLEMENTATION. @transient override lazy val sharedState: TestHiveSharedState = { existingSharedState.getOrElse(