From c44d140cae99d0b880e6d25f158125ad3adc6a05 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Mar 2016 22:21:15 -0700 Subject: [PATCH] Revert "[SPARK-14014][SQL] Replace existing catalog with SessionCatalog" This reverts commit 5dfc01976bb0d72489620b4f32cc12d620bb6260. --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 3 +- project/MimaExcludes.scala | 3 - python/pyspark/sql/context.py | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 20 +- .../spark/sql/catalyst/analysis/Catalog.scala | 218 ++++++++ .../sql/catalyst/analysis/unresolved.scala | 2 +- .../catalyst/catalog/InMemoryCatalog.scala | 35 +- .../sql/catalyst/catalog/SessionCatalog.scala | 123 ++--- .../sql/catalyst/catalog/interface.scala | 2 - .../sql/catalyst/analysis/AnalysisSuite.scala | 6 +- .../sql/catalyst/analysis/AnalysisTest.scala | 23 +- .../analysis/DecimalPrecisionSuite.scala | 25 +- .../catalyst/catalog/CatalogTestCases.scala | 3 +- .../catalog/SessionCatalogSuite.scala | 20 +- .../BooleanSimplificationSuite.scala | 11 +- .../optimizer/EliminateSortsSuite.scala | 5 +- .../org/apache/spark/sql/SQLContext.scala | 73 +-- .../sql/execution/command/commands.scala | 8 +- .../spark/sql/execution/datasources/ddl.scala | 24 +- .../sql/execution/datasources/rules.scala | 10 +- .../spark/sql/internal/SessionState.scala | 7 +- .../apache/spark/sql/ListTablesSuite.scala | 15 +- .../apache/spark/sql/SQLContextSuite.scala | 9 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 22 +- .../parquet/ParquetQuerySuite.scala | 6 +- .../apache/spark/sql/test/SQLTestUtils.scala | 4 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 3 +- .../sql/hive/thriftserver/CliSuite.scala | 5 +- .../apache/spark/sql/hive/HiveCatalog.scala | 5 +- .../apache/spark/sql/hive/HiveContext.scala | 498 ++++++++---------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 60 ++- .../spark/sql/hive/HiveSessionCatalog.scala | 104 ---- .../spark/sql/hive/HiveSessionState.scala | 10 +- .../spark/sql/hive/client/HiveClient.scala | 3 + .../sql/hive/client/HiveClientImpl.scala | 4 + .../hive/execution/CreateTableAsSelect.scala | 4 +- .../hive/execution/CreateViewAsSelect.scala | 4 +- .../hive/execution/InsertIntoHiveTable.scala | 14 +- .../spark/sql/hive/execution/commands.scala | 9 +- .../apache/spark/sql/hive/test/TestHive.scala | 151 ++---- .../hive/JavaMetastoreDataSourcesSuite.java | 5 +- .../spark/sql/hive/HiveContextSuite.scala | 38 -- .../sql/hive/HiveMetastoreCatalogSuite.scala | 9 +- .../spark/sql/hive/ListTablesSuite.scala | 6 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 31 +- .../spark/sql/hive/MultiDatabaseSuite.scala | 5 +- .../spark/sql/hive/StatisticsSuite.scala | 3 +- .../spark/sql/hive/client/VersionsSuite.scala | 4 + .../sql/hive/execution/HiveQuerySuite.scala | 16 +- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- .../spark/sql/hive/orc/OrcQuerySuite.scala | 4 +- .../apache/spark/sql/hive/parquetSuites.scala | 24 +- 52 files changed, 783 insertions(+), 919 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index eef365b42e..63acbadfa6 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1817,8 +1817,7 @@ test_that("approxQuantile() on a DataFrame", { test_that("SQL error message is returned from JVM", { retError <- tryCatch(sql(sqlContext, "select * from blah"), error = function(e) e) - expect_equal(grepl("Table not found", retError), TRUE) - expect_equal(grepl("blah", retError), TRUE) + expect_equal(grepl("Table not found: blah", retError), TRUE) }) irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 915898389c..42eafcb0f5 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -562,9 +562,6 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Logging.initializeLogIfNecessary"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerEvent.logEvent"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance") - ) ++ Seq( - // [SPARK-14014] Replace existing analysis.Catalog with SessionCatalog - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.this") ) ++ Seq( // [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Logging"), diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 4008332c84..9c2f6a3c56 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -554,7 +554,7 @@ class SQLContext(object): >>> sqlContext.registerDataFrameAsTable(df, "table1") >>> "table1" in sqlContext.tableNames() True - >>> "table1" in sqlContext.tableNames("default") + >>> "table1" in sqlContext.tableNames("db") True """ if dbName is None: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 54543eebb7..07b0f5ee70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -24,7 +24,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} -import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -37,22 +36,23 @@ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.types._ /** - * A trivial [[Analyzer]] with an dummy [[SessionCatalog]] and [[EmptyFunctionRegistry]]. - * Used for testing when all relations are already filled in and the analyzer needs only - * to resolve attribute references. + * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing + * when all relations are already filled in and the analyzer needs only to resolve attribute + * references. */ object SimpleAnalyzer - extends SimpleAnalyzer(new SimpleCatalystConf(caseSensitiveAnalysis = true)) -class SimpleAnalyzer(conf: CatalystConf) - extends Analyzer(new SessionCatalog(new InMemoryCatalog, conf), EmptyFunctionRegistry, conf) + extends Analyzer( + EmptyCatalog, + EmptyFunctionRegistry, + new SimpleCatalystConf(caseSensitiveAnalysis = true)) /** * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and - * [[UnresolvedRelation]]s into fully typed objects using information in a - * [[SessionCatalog]] and a [[FunctionRegistry]]. + * [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and + * a [[FunctionRegistry]]. */ class Analyzer( - catalog: SessionCatalog, + catalog: Catalog, registry: FunctionRegistry, conf: CatalystConf, maxIterations: Int = 100) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala new file mode 100644 index 0000000000..2f0a4dbc10 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -0,0 +1,218 @@ +/* + * 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.catalyst.analysis + +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{CatalystConf, EmptyConf, TableIdentifier} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} + + +/** + * An interface for looking up relations by name. Used by an [[Analyzer]]. + */ +trait Catalog { + + val conf: CatalystConf + + def tableExists(tableIdent: TableIdentifier): Boolean + + def lookupRelation(tableIdent: TableIdentifier, alias: Option[String] = None): LogicalPlan + + def setCurrentDatabase(databaseName: String): Unit = { + throw new UnsupportedOperationException + } + + /** + * Returns tuples of (tableName, isTemporary) for all tables in the given database. + * isTemporary is a Boolean value indicates if a table is a temporary or not. + */ + def getTables(databaseName: Option[String]): Seq[(String, Boolean)] + + def refreshTable(tableIdent: TableIdentifier): Unit + + def registerTable(tableIdent: TableIdentifier, plan: LogicalPlan): Unit + + def unregisterTable(tableIdent: TableIdentifier): Unit + + def unregisterAllTables(): Unit + + /** + * Get the table name of TableIdentifier for temporary tables. + */ + protected def getTableName(tableIdent: TableIdentifier): String = { + // It is not allowed to specify database name for temporary tables. + // We check it here and throw exception if database is defined. + if (tableIdent.database.isDefined) { + throw new AnalysisException("Specifying database name or other qualifiers are not allowed " + + "for temporary tables. If the table name has dots (.) in it, please quote the " + + "table name with backticks (`).") + } + if (conf.caseSensitiveAnalysis) { + tableIdent.table + } else { + tableIdent.table.toLowerCase + } + } +} + +class SimpleCatalog(val conf: CatalystConf) extends Catalog { + private[this] val tables = new ConcurrentHashMap[String, LogicalPlan] + + override def registerTable(tableIdent: TableIdentifier, plan: LogicalPlan): Unit = { + tables.put(getTableName(tableIdent), plan) + } + + override def unregisterTable(tableIdent: TableIdentifier): Unit = { + tables.remove(getTableName(tableIdent)) + } + + override def unregisterAllTables(): Unit = { + tables.clear() + } + + override def tableExists(tableIdent: TableIdentifier): Boolean = { + tables.containsKey(getTableName(tableIdent)) + } + + override def lookupRelation( + tableIdent: TableIdentifier, + alias: Option[String] = None): LogicalPlan = { + val tableName = getTableName(tableIdent) + val table = tables.get(tableName) + if (table == null) { + throw new AnalysisException("Table not found: " + tableName) + } + val qualifiedTable = SubqueryAlias(tableName, table) + + // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are + // properly qualified with this alias. + alias + .map(a => SubqueryAlias(a, qualifiedTable)) + .getOrElse(qualifiedTable) + } + + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + tables.keySet().asScala.map(_ -> true).toSeq + } + + override def refreshTable(tableIdent: TableIdentifier): Unit = { + throw new UnsupportedOperationException + } +} + +/** + * A trait that can be mixed in with other Catalogs allowing specific tables to be overridden with + * new logical plans. This can be used to bind query result to virtual tables, or replace tables + * with in-memory cached versions. Note that the set of overrides is stored in memory and thus + * lost when the JVM exits. + */ +trait OverrideCatalog extends Catalog { + private[this] val overrides = new ConcurrentHashMap[String, LogicalPlan] + + private def getOverriddenTable(tableIdent: TableIdentifier): Option[LogicalPlan] = { + if (tableIdent.database.isDefined) { + None + } else { + Option(overrides.get(getTableName(tableIdent))) + } + } + + abstract override def tableExists(tableIdent: TableIdentifier): Boolean = { + getOverriddenTable(tableIdent) match { + case Some(_) => true + case None => super.tableExists(tableIdent) + } + } + + abstract override def lookupRelation( + tableIdent: TableIdentifier, + alias: Option[String] = None): LogicalPlan = { + getOverriddenTable(tableIdent) match { + case Some(table) => + val tableName = getTableName(tableIdent) + val qualifiedTable = SubqueryAlias(tableName, table) + + // If an alias was specified by the lookup, wrap the plan in a sub-query so that attributes + // are properly qualified with this alias. + alias.map(a => SubqueryAlias(a, qualifiedTable)).getOrElse(qualifiedTable) + + case None => super.lookupRelation(tableIdent, alias) + } + } + + abstract override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + overrides.keySet().asScala.map(_ -> true).toSeq ++ super.getTables(databaseName) + } + + override def registerTable(tableIdent: TableIdentifier, plan: LogicalPlan): Unit = { + overrides.put(getTableName(tableIdent), plan) + } + + override def unregisterTable(tableIdent: TableIdentifier): Unit = { + if (tableIdent.database.isEmpty) { + overrides.remove(getTableName(tableIdent)) + } + } + + override def unregisterAllTables(): Unit = { + overrides.clear() + } +} + +/** + * A trivial catalog that returns an error when a relation is requested. Used for testing when all + * relations are already filled in and the analyzer needs only to resolve attribute references. + */ +object EmptyCatalog extends Catalog { + + override val conf: CatalystConf = EmptyConf + + override def tableExists(tableIdent: TableIdentifier): Boolean = { + throw new UnsupportedOperationException + } + + override def lookupRelation( + tableIdent: TableIdentifier, + alias: Option[String] = None): LogicalPlan = { + throw new UnsupportedOperationException + } + + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + throw new UnsupportedOperationException + } + + override def registerTable(tableIdent: TableIdentifier, plan: LogicalPlan): Unit = { + throw new UnsupportedOperationException + } + + override def unregisterTable(tableIdent: TableIdentifier): Unit = { + throw new UnsupportedOperationException + } + + override def unregisterAllTables(): Unit = { + throw new UnsupportedOperationException + } + + override def refreshTable(tableIdent: TableIdentifier): Unit = { + throw new UnsupportedOperationException + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index e73d367a73..9518309fbf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -34,7 +34,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str errors.TreeNodeException(tree, s"Invalid call to $function on unresolved object", null) /** - * Holds the name of a relation that has yet to be looked up in a catalog. + * Holds the name of a relation that has yet to be looked up in a [[Catalog]]. */ case class UnresolvedRelation( tableIdentifier: TableIdentifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index e216fa5528..7ead1ddebe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -52,34 +52,37 @@ class InMemoryCatalog extends ExternalCatalog { names.filter { funcName => regex.pattern.matcher(funcName).matches() } } - private def functionExists(db: String, funcName: String): Boolean = { + private def existsFunction(db: String, funcName: String): Boolean = { requireDbExists(db) catalog(db).functions.contains(funcName) } - private def partitionExists(db: String, table: String, spec: TablePartitionSpec): Boolean = { + private def existsTable(db: String, table: String): Boolean = { + requireDbExists(db) + catalog(db).tables.contains(table) + } + + private def existsPartition(db: String, table: String, spec: TablePartitionSpec): Boolean = { requireTableExists(db, table) catalog(db).tables(table).partitions.contains(spec) } private def requireFunctionExists(db: String, funcName: String): Unit = { - if (!functionExists(db, funcName)) { - throw new AnalysisException( - s"Function not found: '$funcName' does not exist in database '$db'") + if (!existsFunction(db, funcName)) { + throw new AnalysisException(s"Function '$funcName' does not exist in database '$db'") } } private def requireTableExists(db: String, table: String): Unit = { - if (!tableExists(db, table)) { - throw new AnalysisException( - s"Table not found: '$table' does not exist in database '$db'") + if (!existsTable(db, table)) { + throw new AnalysisException(s"Table '$table' does not exist in database '$db'") } } private def requirePartitionExists(db: String, table: String, spec: TablePartitionSpec): Unit = { - if (!partitionExists(db, table, spec)) { + if (!existsPartition(db, table, spec)) { throw new AnalysisException( - s"Partition not found: database '$db' table '$table' does not contain: '$spec'") + s"Partition does not exist in database '$db' table '$table': '$spec'") } } @@ -156,7 +159,7 @@ class InMemoryCatalog extends ExternalCatalog { ignoreIfExists: Boolean): Unit = synchronized { requireDbExists(db) val table = tableDefinition.name.table - if (tableExists(db, table)) { + if (existsTable(db, table)) { if (!ignoreIfExists) { throw new AnalysisException(s"Table '$table' already exists in database '$db'") } @@ -170,7 +173,7 @@ class InMemoryCatalog extends ExternalCatalog { table: String, ignoreIfNotExists: Boolean): Unit = synchronized { requireDbExists(db) - if (tableExists(db, table)) { + if (existsTable(db, table)) { catalog(db).tables.remove(table) } else { if (!ignoreIfNotExists) { @@ -197,17 +200,13 @@ class InMemoryCatalog extends ExternalCatalog { catalog(db).tables(table).table } - override def tableExists(db: String, table: String): Boolean = synchronized { - requireDbExists(db) - catalog(db).tables.contains(table) - } - override def listTables(db: String): Seq[String] = synchronized { requireDbExists(db) catalog(db).tables.keySet.toSeq } override def listTables(db: String, pattern: String): Seq[String] = synchronized { + requireDbExists(db) filterPattern(listTables(db), pattern) } @@ -296,7 +295,7 @@ class InMemoryCatalog extends ExternalCatalog { override def createFunction(db: String, func: CatalogFunction): Unit = synchronized { requireDbExists(db) - if (functionExists(db, func.name.funcName)) { + if (existsFunction(db, func.name.funcName)) { throw new AnalysisException(s"Function '$func' already exists in '$db' database") } else { catalog(db).functions.put(func.name.funcName, func) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 34265faa74..3ac2bcf7e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -22,7 +22,6 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} @@ -32,34 +31,17 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} * proxy to the underlying metastore (e.g. Hive Metastore) and it also manages temporary * tables and functions of the Spark Session that it belongs to. */ -class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { +class SessionCatalog(externalCatalog: ExternalCatalog) { import ExternalCatalog._ - def this(externalCatalog: ExternalCatalog) { - this(externalCatalog, new SimpleCatalystConf(true)) - } - - protected[this] val tempTables = new ConcurrentHashMap[String, LogicalPlan] - protected[this] val tempFunctions = new ConcurrentHashMap[String, CatalogFunction] + private[this] val tempTables = new ConcurrentHashMap[String, LogicalPlan] + private[this] val tempFunctions = new ConcurrentHashMap[String, CatalogFunction] // Note: we track current database here because certain operations do not explicitly // specify the database (e.g. DROP TABLE my_table). In these cases we must first // check whether the temporary table or function exists, then, if not, operate on // the corresponding item in the current database. - protected[this] var currentDb = { - val defaultName = "default" - val defaultDbDefinition = CatalogDatabase(defaultName, "default database", "", Map()) - // Initialize default database if it doesn't already exist - createDatabase(defaultDbDefinition, ignoreIfExists = true) - defaultName - } - - /** - * Format table name, taking into account case sensitivity. - */ - protected[this] def formatTableName(name: String): String = { - if (conf.caseSensitiveAnalysis) name else name.toLowerCase - } + private[this] var currentDb = "default" // ---------------------------------------------------------------------------- // Databases @@ -123,8 +105,8 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def createTable(tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = { val db = tableDefinition.name.database.getOrElse(currentDb) - val table = formatTableName(tableDefinition.name.table) - val newTableDefinition = tableDefinition.copy(name = TableIdentifier(table, Some(db))) + val newTableDefinition = tableDefinition.copy( + name = TableIdentifier(tableDefinition.name.table, Some(db))) externalCatalog.createTable(db, newTableDefinition, ignoreIfExists) } @@ -139,8 +121,8 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def alterTable(tableDefinition: CatalogTable): Unit = { val db = tableDefinition.name.database.getOrElse(currentDb) - val table = formatTableName(tableDefinition.name.table) - val newTableDefinition = tableDefinition.copy(name = TableIdentifier(table, Some(db))) + val newTableDefinition = tableDefinition.copy( + name = TableIdentifier(tableDefinition.name.table, Some(db))) externalCatalog.alterTable(db, newTableDefinition) } @@ -150,8 +132,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def getTable(name: TableIdentifier): CatalogTable = { val db = name.database.getOrElse(currentDb) - val table = formatTableName(name.table) - externalCatalog.getTable(db, table) + externalCatalog.getTable(db, name.table) } // ------------------------------------------------------------- @@ -165,11 +146,10 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { name: String, tableDefinition: LogicalPlan, ignoreIfExists: Boolean): Unit = { - val table = formatTableName(name) - if (tempTables.containsKey(table) && !ignoreIfExists) { + if (tempTables.containsKey(name) && !ignoreIfExists) { throw new AnalysisException(s"Temporary table '$name' already exists.") } - tempTables.put(table, tableDefinition) + tempTables.put(name, tableDefinition) } /** @@ -186,13 +166,11 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { throw new AnalysisException("rename does not support moving tables across databases") } val db = oldName.database.getOrElse(currentDb) - val oldTableName = formatTableName(oldName.table) - val newTableName = formatTableName(newName.table) - if (oldName.database.isDefined || !tempTables.containsKey(oldTableName)) { - externalCatalog.renameTable(db, oldTableName, newTableName) + if (oldName.database.isDefined || !tempTables.containsKey(oldName.table)) { + externalCatalog.renameTable(db, oldName.table, newName.table) } else { - val table = tempTables.remove(oldTableName) - tempTables.put(newTableName, table) + val table = tempTables.remove(oldName.table) + tempTables.put(newName.table, table) } } @@ -205,11 +183,10 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def dropTable(name: TableIdentifier, ignoreIfNotExists: Boolean): Unit = { val db = name.database.getOrElse(currentDb) - val table = formatTableName(name.table) - if (name.database.isDefined || !tempTables.containsKey(table)) { - externalCatalog.dropTable(db, table, ignoreIfNotExists) + if (name.database.isDefined || !tempTables.containsKey(name.table)) { + externalCatalog.dropTable(db, name.table, ignoreIfNotExists) } else { - tempTables.remove(table) + tempTables.remove(name.table) } } @@ -222,42 +199,27 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def lookupRelation(name: TableIdentifier, alias: Option[String] = None): LogicalPlan = { val db = name.database.getOrElse(currentDb) - val table = formatTableName(name.table) val relation = - if (name.database.isDefined || !tempTables.containsKey(table)) { - val metadata = externalCatalog.getTable(db, table) + if (name.database.isDefined || !tempTables.containsKey(name.table)) { + val metadata = externalCatalog.getTable(db, name.table) CatalogRelation(db, metadata, alias) } else { - tempTables.get(table) + tempTables.get(name.table) } - val qualifiedTable = SubqueryAlias(table, relation) + val qualifiedTable = SubqueryAlias(name.table, relation) // If an alias was specified by the lookup, wrap the plan in a subquery so that // attributes are properly qualified with this alias. alias.map(a => SubqueryAlias(a, qualifiedTable)).getOrElse(qualifiedTable) } - /** - * Return whether a table with the specified name exists. - * - * Note: If a database is explicitly specified, then this will return whether the table - * exists in that particular database instead. In that case, even if there is a temporary - * table with the same name, we will return false if the specified database does not - * contain the table. - */ - def tableExists(name: TableIdentifier): Boolean = { - val db = name.database.getOrElse(currentDb) - val table = formatTableName(name.table) - if (name.database.isDefined || !tempTables.containsKey(table)) { - externalCatalog.tableExists(db, table) - } else { - true // it's a temporary table - } - } - /** * List all tables in the specified database, including temporary tables. */ - def listTables(db: String): Seq[TableIdentifier] = listTables(db, "*") + def listTables(db: String): Seq[TableIdentifier] = { + val dbTables = externalCatalog.listTables(db).map { t => TableIdentifier(t, Some(db)) } + val _tempTables = tempTables.keys().asScala.map { t => TableIdentifier(t) } + dbTables ++ _tempTables + } /** * List all matching tables in the specified database, including temporary tables. @@ -272,19 +234,6 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { dbTables ++ _tempTables } - /** - * Refresh the cache entry for a metastore table, if any. - */ - def refreshTable(name: TableIdentifier): Unit = { /* no-op */ } - - /** - * Drop all existing temporary tables. - * For testing only. - */ - def clearTempTables(): Unit = { - tempTables.clear() - } - /** * Return a temporary table exactly as it was stored. * For testing only. @@ -314,8 +263,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { val db = tableName.database.getOrElse(currentDb) - val table = formatTableName(tableName.table) - externalCatalog.createPartitions(db, table, parts, ignoreIfExists) + externalCatalog.createPartitions(db, tableName.table, parts, ignoreIfExists) } /** @@ -327,8 +275,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = { val db = tableName.database.getOrElse(currentDb) - val table = formatTableName(tableName.table) - externalCatalog.dropPartitions(db, table, parts, ignoreIfNotExists) + externalCatalog.dropPartitions(db, tableName.table, parts, ignoreIfNotExists) } /** @@ -342,8 +289,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { val db = tableName.database.getOrElse(currentDb) - val table = formatTableName(tableName.table) - externalCatalog.renamePartitions(db, table, specs, newSpecs) + externalCatalog.renamePartitions(db, tableName.table, specs, newSpecs) } /** @@ -357,8 +303,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def alterPartitions(tableName: TableIdentifier, parts: Seq[CatalogTablePartition]): Unit = { val db = tableName.database.getOrElse(currentDb) - val table = formatTableName(tableName.table) - externalCatalog.alterPartitions(db, table, parts) + externalCatalog.alterPartitions(db, tableName.table, parts) } /** @@ -367,8 +312,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def getPartition(tableName: TableIdentifier, spec: TablePartitionSpec): CatalogTablePartition = { val db = tableName.database.getOrElse(currentDb) - val table = formatTableName(tableName.table) - externalCatalog.getPartition(db, table, spec) + externalCatalog.getPartition(db, tableName.table, spec) } /** @@ -377,8 +321,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) { */ def listPartitions(tableName: TableIdentifier): Seq[CatalogTablePartition] = { val db = tableName.database.getOrElse(currentDb) - val table = formatTableName(tableName.table) - externalCatalog.listPartitions(db, table) + externalCatalog.listPartitions(db, tableName.table) } // ---------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 34803133f6..c4e49614c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -91,8 +91,6 @@ abstract class ExternalCatalog { def getTable(db: String, table: String): CatalogTable - def tableExists(db: String, table: String): Boolean - def listTables(db: String): Seq[String] def listTables(db: String, pattern: String): Seq[String] diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index afc2f327df..8b568b6dd6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -161,10 +161,14 @@ class AnalysisSuite extends AnalysisTest { } test("resolve relations") { - assertAnalysisError(UnresolvedRelation(TableIdentifier("tAbLe"), None), Seq()) + assertAnalysisError( + UnresolvedRelation(TableIdentifier("tAbLe"), None), Seq("Table not found: tAbLe")) + checkAnalysis(UnresolvedRelation(TableIdentifier("TaBlE"), None), testRelation) + checkAnalysis( UnresolvedRelation(TableIdentifier("tAbLe"), None), testRelation, caseSensitive = false) + checkAnalysis( UnresolvedRelation(TableIdentifier("TaBlE"), None), testRelation, caseSensitive = false) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 6fa4beed99..39166c4f8e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -18,21 +18,26 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.SimpleCatalystConf -import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.{SimpleCatalystConf, TableIdentifier} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ trait AnalysisTest extends PlanTest { - protected val caseSensitiveAnalyzer = makeAnalyzer(caseSensitive = true) - protected val caseInsensitiveAnalyzer = makeAnalyzer(caseSensitive = false) + val (caseSensitiveAnalyzer, caseInsensitiveAnalyzer) = { + val caseSensitiveConf = new SimpleCatalystConf(caseSensitiveAnalysis = true) + val caseInsensitiveConf = new SimpleCatalystConf(caseSensitiveAnalysis = false) - private def makeAnalyzer(caseSensitive: Boolean): Analyzer = { - val conf = new SimpleCatalystConf(caseSensitive) - val catalog = new SessionCatalog(new InMemoryCatalog, conf) - catalog.createTempTable("TaBlE", TestRelations.testRelation, ignoreIfExists = true) - new Analyzer(catalog, EmptyFunctionRegistry, conf) { + val caseSensitiveCatalog = new SimpleCatalog(caseSensitiveConf) + val caseInsensitiveCatalog = new SimpleCatalog(caseInsensitiveConf) + + caseSensitiveCatalog.registerTable(TableIdentifier("TaBlE"), TestRelations.testRelation) + caseInsensitiveCatalog.registerTable(TableIdentifier("TaBlE"), TestRelations.testRelation) + + new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitiveConf) { + override val extendedResolutionRules = EliminateSubqueryAliases :: Nil + } -> + new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseInsensitiveConf) { override val extendedResolutionRules = EliminateSubqueryAliases :: Nil } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index 31501864a8..9aa685e1e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.catalyst.SimpleCatalystConf -import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.{SimpleCatalystConf, TableIdentifier} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -31,11 +30,11 @@ import org.apache.spark.sql.types._ class DecimalPrecisionSuite extends PlanTest with BeforeAndAfter { - private val conf = new SimpleCatalystConf(caseSensitiveAnalysis = true) - private val catalog = new SessionCatalog(new InMemoryCatalog, conf) - private val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, conf) + val conf = new SimpleCatalystConf(caseSensitiveAnalysis = true) + val catalog = new SimpleCatalog(conf) + val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, conf) - private val relation = LocalRelation( + val relation = LocalRelation( AttributeReference("i", IntegerType)(), AttributeReference("d1", DecimalType(2, 1))(), AttributeReference("d2", DecimalType(5, 2))(), @@ -44,15 +43,15 @@ class DecimalPrecisionSuite extends PlanTest with BeforeAndAfter { AttributeReference("b", DoubleType)() ) - private val i: Expression = UnresolvedAttribute("i") - private val d1: Expression = UnresolvedAttribute("d1") - private val d2: Expression = UnresolvedAttribute("d2") - private val u: Expression = UnresolvedAttribute("u") - private val f: Expression = UnresolvedAttribute("f") - private val b: Expression = UnresolvedAttribute("b") + val i: Expression = UnresolvedAttribute("i") + val d1: Expression = UnresolvedAttribute("d1") + val d2: Expression = UnresolvedAttribute("d2") + val u: Expression = UnresolvedAttribute("u") + val f: Expression = UnresolvedAttribute("f") + val b: Expression = UnresolvedAttribute("b") before { - catalog.createTempTable("table", relation, ignoreIfExists = true) + catalog.registerTable(TableIdentifier("table"), relation) } private def checkType(expression: Expression, expectedType: DataType): Unit = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogTestCases.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogTestCases.scala index 277c2d717e..a1ea61920d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogTestCases.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogTestCases.scala @@ -225,14 +225,13 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { test("list tables without pattern") { val catalog = newBasicCatalog() - intercept[AnalysisException] { catalog.listTables("unknown_db") } assert(catalog.listTables("db1").toSet == Set.empty) assert(catalog.listTables("db2").toSet == Set("tbl1", "tbl2")) } test("list tables with pattern") { val catalog = newBasicCatalog() - intercept[AnalysisException] { catalog.listTables("unknown_db", "*") } + intercept[AnalysisException] { catalog.listTables("unknown_db") } assert(catalog.listTables("db1", "*").toSet == Set.empty) assert(catalog.listTables("db2", "*").toSet == Set("tbl1", "tbl2")) assert(catalog.listTables("db2", "tbl*").toSet == Set("tbl1", "tbl2")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 74e995cc5b..e1973ee258 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -397,24 +397,6 @@ class SessionCatalogSuite extends SparkFunSuite { TableIdentifier("tbl1", Some("db2")), alias = Some(alias)) == relationWithAlias) } - test("table exists") { - val catalog = new SessionCatalog(newBasicCatalog()) - assert(catalog.tableExists(TableIdentifier("tbl1", Some("db2")))) - assert(catalog.tableExists(TableIdentifier("tbl2", Some("db2")))) - assert(!catalog.tableExists(TableIdentifier("tbl3", Some("db2")))) - assert(!catalog.tableExists(TableIdentifier("tbl1", Some("db1")))) - assert(!catalog.tableExists(TableIdentifier("tbl2", Some("db1")))) - // If database is explicitly specified, do not check temporary tables - val tempTable = Range(1, 10, 1, 10, Seq()) - catalog.createTempTable("tbl3", tempTable, ignoreIfExists = false) - assert(!catalog.tableExists(TableIdentifier("tbl3", Some("db2")))) - // If database is not explicitly specified, check the current database - catalog.setCurrentDatabase("db2") - assert(catalog.tableExists(TableIdentifier("tbl1"))) - assert(catalog.tableExists(TableIdentifier("tbl2"))) - assert(catalog.tableExists(TableIdentifier("tbl3"))) - } - test("list tables without pattern") { val catalog = new SessionCatalog(newBasicCatalog()) val tempTable = Range(1, 10, 2, 10, Seq()) @@ -447,7 +429,7 @@ class SessionCatalogSuite extends SparkFunSuite { assert(catalog.listTables("db2", "*1").toSet == Set(TableIdentifier("tbl1"), TableIdentifier("tbl1", Some("db2")))) intercept[AnalysisException] { - catalog.listTables("unknown_db", "*") + catalog.listTables("unknown_db") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index e2c76b700f..2ab31eea8a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.SimpleCatalystConf import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ @@ -138,11 +137,11 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { checkCondition(!(('a || 'b) && ('c || 'd)), (!'a && !'b) || (!'c && !'d)) } - private val caseInsensitiveConf = new SimpleCatalystConf(false) - private val caseInsensitiveAnalyzer = new Analyzer( - new SessionCatalog(new InMemoryCatalog, caseInsensitiveConf), - EmptyFunctionRegistry, - caseInsensitiveConf) + private val caseInsensitiveAnalyzer = + new Analyzer( + EmptyCatalog, + EmptyFunctionRegistry, + new SimpleCatalystConf(caseSensitiveAnalysis = false)) test("(a && b) || (a && c) => a && (b || c) when case insensitive") { val plan = caseInsensitiveAnalyzer.execute( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala index 3824c67563..a4c8d1c6d2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.SimpleCatalystConf -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} -import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry, SimpleCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ @@ -29,7 +28,7 @@ import org.apache.spark.sql.catalyst.rules._ class EliminateSortsSuite extends PlanTest { val conf = new SimpleCatalystConf(caseSensitiveAnalysis = true, orderByOrdinal = false) - val catalog = new SessionCatalog(new InMemoryCatalog, conf) + val catalog = new SimpleCatalog(conf) val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, conf) object Optimize extends RuleExecutor[LogicalPlan] { 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 e413e77bc1..853a74c827 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 @@ -25,14 +25,13 @@ import scala.collection.JavaConverters._ import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} @@ -66,14 +65,13 @@ 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) + val isRootContext: Boolean) extends Logging with Serializable { self => - def this(sc: SparkContext) = { - this(sc, new CacheManager, SQLContext.createListenerAndUI(sc), true, new InMemoryCatalog) + def this(sparkContext: SparkContext) = { + this(sparkContext, new CacheManager, SQLContext.createListenerAndUI(sparkContext), true) } def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) @@ -111,8 +109,7 @@ class SQLContext private[sql]( sparkContext = sparkContext, cacheManager = cacheManager, listener = listener, - isRootContext = false, - externalCatalog = externalCatalog) + isRootContext = false) } /** @@ -189,12 +186,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 parseSql(sql: String): LogicalPlan = sessionState.sqlParser.parsePlan(sql) protected[sql] def executeSql(sql: String): QueryExecution = executePlan(parseSql(sql)) @@ -208,6 +199,30 @@ class SQLContext private[sql]( sparkContext.addJar(path) } + { + // We extract spark sql settings from SparkContext's conf and put them to + // Spark SQL's conf. + // First, we populate the SQLConf (conf). So, we can make sure that other values using + // those settings in their construction can get the correct settings. + // For example, metadataHive in HiveContext may need both spark.sql.hive.metastore.version + // and spark.sql.hive.metastore.jars to get correctly constructed. + val properties = new Properties + sparkContext.getConf.getAll.foreach { + case (key, value) if key.startsWith("spark.sql") => properties.setProperty(key, value) + case _ => + } + // We directly put those settings to conf to avoid of calling setConf, which may have + // side-effects. For example, in HiveContext, setConf may cause executionHive and metadataHive + // get constructed. If we call setConf directly, the constructed metadataHive may have + // wrong settings, or the construction may fail. + conf.setConf(properties) + // After we have populated SQLConf, we call setConf to populate other confs in the subclass + // (e.g. hiveconf in HiveContext). + properties.asScala.foreach { + case (key, value) => setConf(key, value) + } + } + /** * :: Experimental :: * A collection of methods that are considered experimental, but can be used to hook into @@ -668,10 +683,8 @@ class SQLContext private[sql]( * only during the lifetime of this instance of SQLContext. */ private[sql] def registerDataFrameAsTable(df: DataFrame, tableName: String): Unit = { - sessionState.catalog.createTempTable( - sessionState.sqlParser.parseTableIdentifier(tableName).table, - df.logicalPlan, - ignoreIfExists = true) + sessionState.catalog.registerTable( + sessionState.sqlParser.parseTableIdentifier(tableName), df.logicalPlan) } /** @@ -684,7 +697,7 @@ class SQLContext private[sql]( */ def dropTempTable(tableName: String): Unit = { cacheManager.tryUncacheQuery(table(tableName)) - sessionState.catalog.dropTable(TableIdentifier(tableName), ignoreIfNotExists = true) + sessionState.catalog.unregisterTable(TableIdentifier(tableName)) } /** @@ -811,7 +824,9 @@ class SQLContext private[sql]( * @since 1.3.0 */ def tableNames(): Array[String] = { - tableNames(sessionState.catalog.getCurrentDatabase) + sessionState.catalog.getTables(None).map { + case (tableName, _) => tableName + }.toArray } /** @@ -821,7 +836,9 @@ class SQLContext private[sql]( * @since 1.3.0 */ def tableNames(databaseName: String): Array[String] = { - sessionState.catalog.listTables(databaseName).map(_.table).toArray + sessionState.catalog.getTables(Some(databaseName)).map { + case (tableName, _) => tableName + }.toArray } @transient @@ -1008,18 +1025,4 @@ object SQLContext { } sqlListener.get() } - - /** - * Extract `spark.sql.*` properties from the conf and return them as a [[Properties]]. - */ - private[sql] def getSQLProperties(sparkConf: SparkConf): Properties = { - val properties = new Properties - sparkConf.getAll.foreach { case (key, value) => - if (key.startsWith("spark.sql")) { - properties.setProperty(key, value) - } - } - properties - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 964f0a7a7b..59c3ffcf48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -339,12 +339,10 @@ case class ShowTablesCommand(databaseName: Option[String]) extends RunnableComma override def run(sqlContext: SQLContext): Seq[Row] = { // Since we need to return a Seq of rows, we will call getTables directly // instead of calling tables in sqlContext. - val catalog = sqlContext.sessionState.catalog - val db = databaseName.getOrElse(catalog.getCurrentDatabase) - val rows = catalog.listTables(db).map { t => - val isTemp = t.database.isEmpty - Row(t.table, isTemp) + val rows = sqlContext.sessionState.catalog.getTables(databaseName).map { + case (tableName, isTemporary) => Row(tableName, isTemporary) } + rows } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 24923bbb10..9e8e0352db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -93,21 +93,15 @@ case class CreateTempTableUsing( provider: String, options: Map[String, String]) extends RunnableCommand { - if (tableIdent.database.isDefined) { - throw new AnalysisException( - s"Temporary table '$tableIdent' should not have specified a database") - } - def run(sqlContext: SQLContext): Seq[Row] = { val dataSource = DataSource( sqlContext, userSpecifiedSchema = userSpecifiedSchema, className = provider, options = options) - sqlContext.sessionState.catalog.createTempTable( - tableIdent.table, - Dataset.ofRows(sqlContext, LogicalRelation(dataSource.resolveRelation())).logicalPlan, - ignoreIfExists = true) + sqlContext.sessionState.catalog.registerTable( + tableIdent, + Dataset.ofRows(sqlContext, LogicalRelation(dataSource.resolveRelation())).logicalPlan) Seq.empty[Row] } @@ -121,11 +115,6 @@ case class CreateTempTableUsingAsSelect( options: Map[String, String], query: LogicalPlan) extends RunnableCommand { - if (tableIdent.database.isDefined) { - throw new AnalysisException( - s"Temporary table '$tableIdent' should not have specified a database") - } - override def run(sqlContext: SQLContext): Seq[Row] = { val df = Dataset.ofRows(sqlContext, query) val dataSource = DataSource( @@ -135,10 +124,9 @@ case class CreateTempTableUsingAsSelect( bucketSpec = None, options = options) val result = dataSource.write(mode, df) - sqlContext.sessionState.catalog.createTempTable( - tableIdent.table, - Dataset.ofRows(sqlContext, LogicalRelation(result)).logicalPlan, - ignoreIfExists = true) + sqlContext.sessionState.catalog.registerTable( + tableIdent, + Dataset.ofRows(sqlContext, LogicalRelation(result)).logicalPlan) Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 28ac4583e9..63f0e4f8c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -19,12 +19,10 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, HadoopFsRelation, InsertableRelation} /** @@ -101,9 +99,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { /** * A rule to do various checks before inserting into or writing to a data source table. */ -private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) - extends (LogicalPlan => Unit) { - +private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => Unit) { def failAnalysis(msg: String): Unit = { throw new AnalysisException(msg) } def apply(plan: LogicalPlan): Unit = { @@ -143,7 +139,7 @@ private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } PartitioningUtils.validatePartitionColumnDataTypes( - r.schema, part.keySet.toSeq, conf.caseSensitiveAnalysis) + r.schema, part.keySet.toSeq, catalog.conf.caseSensitiveAnalysis) // Get all input data source relations of the query. val srcRelations = query.collect { @@ -194,7 +190,7 @@ private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } PartitioningUtils.validatePartitionColumnDataTypes( - c.child.schema, c.partitionColumns, conf.caseSensitiveAnalysis) + c.child.schema, c.partitionColumns, catalog.conf.caseSensitiveAnalysis) for { spec <- c.bucketSpec 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 e5f02caabc..e6be0ab3bc 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 @@ -18,8 +18,7 @@ package org.apache.spark.sql.internal 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.analysis.{Analyzer, Catalog, FunctionRegistry, SimpleCatalog} import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -46,7 +45,7 @@ private[sql] class SessionState(ctx: SQLContext) { /** * Internal catalog for managing table and database states. */ - lazy val catalog = new SessionCatalog(ctx.externalCatalog, conf) + lazy val catalog: Catalog = new SimpleCatalog(conf) /** * Internal catalog for managing functions registered by the user. @@ -69,7 +68,7 @@ private[sql] class SessionState(ctx: SQLContext) { DataSourceAnalysis :: (if (conf.runSQLOnFile) new ResolveDataSource(ctx) :: Nil else Nil) - override val extendedCheckRules = Seq(datasources.PreWriteCheck(conf, catalog)) + override val extendedCheckRules = Seq(datasources.PreWriteCheck(catalog)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index bb54c525cb..2820e4fa23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -33,8 +33,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex } after { - sqlContext.sessionState.catalog.dropTable( - TableIdentifier("ListTablesSuiteTable"), ignoreIfNotExists = true) + sqlContext.sessionState.catalog.unregisterTable(TableIdentifier("ListTablesSuiteTable")) } test("get all tables") { @@ -46,22 +45,20 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex sql("SHOW tables").filter("tableName = 'ListTablesSuiteTable'"), Row("ListTablesSuiteTable", true)) - sqlContext.sessionState.catalog.dropTable( - TableIdentifier("ListTablesSuiteTable"), ignoreIfNotExists = true) + sqlContext.sessionState.catalog.unregisterTable(TableIdentifier("ListTablesSuiteTable")) assert(sqlContext.tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) } - test("getting all tables with a database name has no impact on returned table names") { + test("getting all Tables with a database name has no impact on returned table names") { checkAnswer( - sqlContext.tables("default").filter("tableName = 'ListTablesSuiteTable'"), + sqlContext.tables("DB").filter("tableName = 'ListTablesSuiteTable'"), Row("ListTablesSuiteTable", true)) checkAnswer( - sql("show TABLES in default").filter("tableName = 'ListTablesSuiteTable'"), + sql("show TABLES in DB").filter("tableName = 'ListTablesSuiteTable'"), Row("ListTablesSuiteTable", true)) - sqlContext.sessionState.catalog.dropTable( - TableIdentifier("ListTablesSuiteTable"), ignoreIfNotExists = true) + sqlContext.sessionState.catalog.unregisterTable(TableIdentifier("ListTablesSuiteTable")) assert(sqlContext.tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 2f62ad4850..2ad92b52c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf -class SQLContextSuite extends SparkFunSuite with SharedSparkContext { +class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ object DummyRule extends Rule[LogicalPlan] { def apply(p: LogicalPlan): LogicalPlan = p @@ -78,11 +78,4 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { sqlContext.experimental.extraOptimizations = Seq(DummyRule) assert(sqlContext.sessionState.optimizer.batches.flatMap(_.rules).contains(DummyRule)) } - - test("SQLContext can access `spark.sql.*` configs") { - sc.conf.set("spark.sql.with.or.without.you", "my love") - val sqlContext = new SQLContext(sc) - assert(sqlContext.getConf("spark.sql.with.or.without.you") == "my love") - } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4f36b1b42a..eb486a135f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1397,16 +1397,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-4699 case sensitivity SQL query") { - val orig = sqlContext.getConf(SQLConf.CASE_SENSITIVE) - try { - sqlContext.setConf(SQLConf.CASE_SENSITIVE, false) - val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil - val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF().registerTempTable("testTable1") - checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) - } finally { - sqlContext.setConf(SQLConf.CASE_SENSITIVE, orig) - } + sqlContext.setConf(SQLConf.CASE_SENSITIVE, false) + val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil + val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) + rdd.toDF().registerTempTable("testTable1") + checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) + sqlContext.setConf(SQLConf.CASE_SENSITIVE, true) } test("SPARK-6145: ORDER BY test for nested fields") { @@ -1680,8 +1676,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .format("parquet") .save(path) - // We don't support creating a temporary table while specifying a database - intercept[AnalysisException] { + val message = intercept[AnalysisException] { sqlContext.sql( s""" |CREATE TEMPORARY TABLE db.t @@ -1691,8 +1686,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { |) """.stripMargin) }.getMessage + assert(message.contains("Specifying database name or other qualifiers are not allowed")) - // If you use backticks to quote the name then it's OK. + // If you use backticks to quote the name of a temporary table having dot in it. sqlContext.sql( s""" |CREATE TEMPORARY TABLE `db.t` diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 2f806ebba6..f8166c7ddc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -51,8 +51,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext sql("INSERT INTO TABLE t SELECT * FROM tmp") checkAnswer(sqlContext.table("t"), (data ++ data).map(Row.fromTuple)) } - sqlContext.sessionState.catalog.dropTable( - TableIdentifier("tmp"), ignoreIfNotExists = true) + sqlContext.sessionState.catalog.unregisterTable(TableIdentifier("tmp")) } test("overwriting") { @@ -62,8 +61,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(sqlContext.table("t"), data.map(Row.fromTuple)) } - sqlContext.sessionState.catalog.dropTable( - TableIdentifier("tmp"), ignoreIfNotExists = true) + sqlContext.sessionState.catalog.unregisterTable(TableIdentifier("tmp")) } test("self-join") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 80a85a6615..d48358566e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -189,8 +189,8 @@ private[sql] trait SQLTestUtils * `f` returns. */ protected def activateDatabase(db: String)(f: => Unit): Unit = { - sqlContext.sessionState.catalog.setCurrentDatabase(db) - try f finally sqlContext.sessionState.catalog.setCurrentDatabase("default") + sqlContext.sql(s"USE $db") + try f finally sqlContext.sql(s"USE default") } /** diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 57693284b0..7fe31b0025 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -150,8 +150,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { } if (sessionState.database != null) { - SparkSQLEnv.hiveContext.sessionState.catalog.setCurrentDatabase( - s"${sessionState.database}") + SparkSQLEnv.hiveContext.runSqlHive(s"USE ${sessionState.database}") } // Execute -i init files (always in silent mode) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 8e1ebe2937..032965d0d9 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -193,7 +193,10 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { ) runCliWithin(2.minute, Seq("--database", "hive_test_db", "-e", "SHOW TABLES;"))( - "" -> "hive_test" + "" + -> "OK", + "" + -> "hive_test" ) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala index 0722fb02a8..491f2aebb4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala @@ -85,6 +85,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends ExternalCatalog wit withClient { getTable(db, table) } } + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -181,10 +182,6 @@ private[spark] class HiveCatalog(client: HiveClient) extends ExternalCatalog wit client.getTable(db, table) } - override def tableExists(db: String, table: String): Boolean = withClient { - client.getTableOption(db, table).isDefined - } - override def listTables(db: String): Seq[String] = withClient { requireDbExists(db) client.listTables(db) 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 ca3ce43591..914f8e9a98 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 @@ -28,7 +28,6 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import scala.language.implicitConversions -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.common.`type`.HiveDecimal @@ -39,7 +38,7 @@ import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} import org.apache.hadoop.util.VersionInfo -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql._ @@ -53,7 +52,6 @@ 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.{DescribeHiveTableCommand, HiveNativeCommand} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.SQLConfEntry import org.apache.spark.sql.internal.SQLConf.SQLConfEntry._ import org.apache.spark.sql.types._ @@ -69,7 +67,7 @@ private[hive] case class CurrentDatabase(ctx: HiveContext) override def foldable: Boolean = true override def nullable: Boolean = false override def eval(input: InternalRow): Any = { - UTF8String.fromString(ctx.sessionState.catalog.getCurrentDatabase) + UTF8String.fromString(ctx.metadataHive.currentDatabase) } } @@ -83,31 +81,15 @@ 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: HiveCatalog) - extends SQLContext(sc, cacheManager, listener, isRootContext, hiveCatalog) with Logging { + @transient private val execHive: HiveClientImpl, + @transient private val metaHive: HiveClient, + isRootContext: Boolean) + extends SQLContext(sc, cacheManager, listener, isRootContext) with Logging { self => - private def this(sc: SparkContext, execHive: HiveClientImpl, metaHive: HiveClient) { - this( - sc, - new CacheManager, - SQLContext.createListenerAndUI(sc), - execHive, - metaHive, - true, - new HiveCatalog(metaHive)) - } - def this(sc: SparkContext) = { - this( - sc, - HiveContext.newClientForExecution(sc.conf, sc.hadoopConfiguration), - HiveContext.newClientForMetadata(sc.conf, sc.hadoopConfiguration)) + this(sc, new CacheManager, SQLContext.createListenerAndUI(sc), null, null, true) } - def this(sc: JavaSparkContext) = this(sc.sc) import org.apache.spark.sql.hive.HiveContext._ @@ -124,10 +106,9 @@ class HiveContext private[hive]( sc = sc, cacheManager = cacheManager, listener = listener, - executionHive = executionHive.newSession(), - metadataHive = metadataHive.newSession(), - isRootContext = false, - hiveCatalog = hiveCatalog) + execHive = executionHive.newSession(), + metaHive = metadataHive.newSession(), + isRootContext = false) } @transient @@ -168,6 +149,41 @@ class HiveContext private[hive]( */ protected[sql] def convertCTAS: Boolean = getConf(CONVERT_CTAS) + /** + * The version of the hive client that will be used to communicate with the metastore. Note that + * this does not necessarily need to be the same version of Hive that is used internally by + * Spark SQL for execution. + */ + protected[hive] def hiveMetastoreVersion: String = getConf(HIVE_METASTORE_VERSION) + + /** + * The location of the jars that should be used to instantiate the HiveMetastoreClient. This + * property can be one of three options: + * - a classpath in the standard format for both hive and hadoop. + * - builtin - attempt to discover the jars that were used to load Spark SQL and use those. This + * option is only valid when using the execution version of Hive. + * - maven - download the correct version of hive on demand from maven. + */ + protected[hive] def hiveMetastoreJars: String = getConf(HIVE_METASTORE_JARS) + + /** + * A comma separated list of class prefixes that should be loaded using the classloader that + * is shared between Spark SQL and a specific version of Hive. An example of classes that should + * be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need + * to be shared are those that interact with classes that are already shared. For example, + * custom appenders that are used by log4j. + */ + protected[hive] def hiveMetastoreSharedPrefixes: Seq[String] = + getConf(HIVE_METASTORE_SHARED_PREFIXES).filterNot(_ == "") + + /** + * A comma separated list of class prefixes that should explicitly be reloaded for each version + * of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a + * prefix that typically would be shared (i.e. org.apache.spark.*) + */ + protected[hive] def hiveMetastoreBarrierPrefixes: Seq[String] = + getConf(HIVE_METASTORE_BARRIER_PREFIXES).filterNot(_ == "") + /* * hive thrift server use background spark sql thread pool to execute sql queries */ @@ -179,6 +195,29 @@ class HiveContext private[hive]( @transient protected[sql] lazy val substitutor = new VariableSubstitution() + /** + * The copy of the hive client that is used for execution. Currently this must always be + * Hive 13 as this is the version of Hive that is packaged with Spark SQL. This copy of the + * client is used for execution related tasks like registering temporary functions or ensuring + * that the ThreadLocal SessionState is correctly populated. This copy of Hive is *not* used + * for storing persistent metadata, and only point to a dummy metastore in a temporary directory. + */ + @transient + protected[hive] lazy val executionHive: HiveClientImpl = if (execHive != null) { + execHive + } else { + logInfo(s"Initializing execution hive, version $hiveExecutionVersion") + val loader = new IsolatedClientLoader( + version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), + sparkConf = sc.conf, + execJars = Seq(), + hadoopConf = sc.hadoopConfiguration, + config = newTemporaryConfiguration(useInMemoryDerby = true), + isolationOn = false, + baseClassLoader = Utils.getContextOrSparkClassLoader) + loader.createClient().asInstanceOf[HiveClientImpl] + } + /** * Overrides default Hive configurations to avoid breaking changes to Spark SQL users. * - allow SQL11 keywords to be used as identifiers @@ -189,6 +228,111 @@ class HiveContext private[hive]( defaultOverrides() + /** + * The copy of the Hive client that is used to retrieve metadata from the Hive MetaStore. + * The version of the Hive client that is used here must match the metastore that is configured + * in the hive-site.xml file. + */ + @transient + protected[hive] lazy val metadataHive: HiveClient = if (metaHive != null) { + metaHive + } else { + val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) + + // We instantiate a HiveConf here to read in the hive-site.xml file and then pass the options + // into the isolated client loader + val metadataConf = new HiveConf(sc.hadoopConfiguration, classOf[HiveConf]) + + val defaultWarehouseLocation = metadataConf.get("hive.metastore.warehouse.dir") + logInfo("default warehouse location is " + defaultWarehouseLocation) + + // `configure` goes second to override other settings. + val allConfig = metadataConf.asScala.map(e => e.getKey -> e.getValue).toMap ++ configure + + val isolatedLoader = if (hiveMetastoreJars == "builtin") { + if (hiveExecutionVersion != hiveMetastoreVersion) { + throw new IllegalArgumentException( + "Builtin jars can only be used when hive execution version == hive metastore version. " + + s"Execution: ${hiveExecutionVersion} != Metastore: ${hiveMetastoreVersion}. " + + "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + + s"or change ${HIVE_METASTORE_VERSION.key} to $hiveExecutionVersion.") + } + + // We recursively find all jars in the class loader chain, + // starting from the given classLoader. + def allJars(classLoader: ClassLoader): Array[URL] = classLoader match { + case null => Array.empty[URL] + case urlClassLoader: URLClassLoader => + urlClassLoader.getURLs ++ allJars(urlClassLoader.getParent) + case other => allJars(other.getParent) + } + + val classLoader = Utils.getContextOrSparkClassLoader + val jars = allJars(classLoader) + if (jars.length == 0) { + throw new IllegalArgumentException( + "Unable to locate hive jars to connect to metastore. " + + "Please set spark.sql.hive.metastore.jars.") + } + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") + new IsolatedClientLoader( + version = metaVersion, + sparkConf = sc.conf, + execJars = jars.toSeq, + hadoopConf = sc.hadoopConfiguration, + config = allConfig, + isolationOn = true, + barrierPrefixes = hiveMetastoreBarrierPrefixes, + sharedPrefixes = hiveMetastoreSharedPrefixes) + } else if (hiveMetastoreJars == "maven") { + // TODO: Support for loading the jars from an already downloaded location. + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using maven.") + IsolatedClientLoader.forVersion( + hiveMetastoreVersion = hiveMetastoreVersion, + hadoopVersion = VersionInfo.getVersion, + sparkConf = sc.conf, + hadoopConf = sc.hadoopConfiguration, + config = allConfig, + barrierPrefixes = hiveMetastoreBarrierPrefixes, + sharedPrefixes = hiveMetastoreSharedPrefixes) + } else { + // Convert to files and expand any directories. + val jars = + hiveMetastoreJars + .split(File.pathSeparator) + .flatMap { + case path if new File(path).getName() == "*" => + val files = new File(path).getParentFile().listFiles() + if (files == null) { + logWarning(s"Hive jar path '$path' does not exist.") + Nil + } else { + files.filter(_.getName().toLowerCase().endsWith(".jar")) + } + case path => + new File(path) :: Nil + } + .map(_.toURI.toURL) + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion " + + s"using ${jars.mkString(":")}") + new IsolatedClientLoader( + version = metaVersion, + sparkConf = sc.conf, + execJars = jars.toSeq, + hadoopConf = sc.hadoopConfiguration, + config = allConfig, + isolationOn = true, + barrierPrefixes = hiveMetastoreBarrierPrefixes, + sharedPrefixes = hiveMetastoreSharedPrefixes) + } + isolatedLoader.createClient() + } + protected[sql] override def parseSql(sql: String): LogicalPlan = { executionHive.withHiveState { super.parseSql(substitutor.substitute(hiveconf, sql)) @@ -288,7 +432,7 @@ class HiveContext private[hive]( // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - sessionState.catalog.alterTable( + sessionState.catalog.client.alterTable( relation.table.copy( properties = relation.table.properties + (StatsSetupConst.TOTAL_SIZE -> newTotalSize.toString))) @@ -315,10 +459,64 @@ class HiveContext private[hive]( setConf(entry.key, entry.stringConverter(value)) } + /** Overridden by child classes that need to set configuration before the client init. */ + protected def configure(): Map[String, String] = { + // Hive 0.14.0 introduces timeout operations in HiveConf, and changes default values of a bunch + // of time `ConfVar`s by adding time suffixes (`s`, `ms`, and `d` etc.). This breaks backwards- + // compatibility when users are trying to connecting to a Hive metastore of lower version, + // because these options are expected to be integral values in lower versions of Hive. + // + // Here we enumerate all time `ConfVar`s and convert their values to numeric strings according + // to their output time units. + Seq( + ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY -> TimeUnit.SECONDS, + ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME -> TimeUnit.SECONDS, + ConfVars.HMSHANDLERINTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.METASTORE_EVENT_DB_LISTENER_TTL -> TimeUnit.SECONDS, + ConfVars.METASTORE_EVENT_CLEAN_FREQ -> TimeUnit.SECONDS, + ConfVars.METASTORE_EVENT_EXPIRY_DURATION -> TimeUnit.SECONDS, + ConfVars.METASTORE_AGGREGATE_STATS_CACHE_TTL -> TimeUnit.SECONDS, + ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT -> TimeUnit.MILLISECONDS, + ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT -> TimeUnit.MILLISECONDS, + ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_STATS_JDBC_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_STATS_RETRIES_WAIT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES -> TimeUnit.SECONDS, + ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_TXN_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_COMPACTOR_WORKER_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_COMPACTOR_CHECK_INTERVAL -> TimeUnit.SECONDS, + ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.SERVER_READ_SOCKET_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.SPARK_JOB_MONITOR_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT -> TimeUnit.MILLISECONDS + ).map { case (confVar, unit) => + confVar.varname -> hiveconf.getTimeVar(confVar, unit).toString + }.toMap + } + /** * SQLConf and HiveConf contracts: * - * 1. create a new o.a.h.hive.ql.session.SessionState for each HiveContext + * 1. create a new 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. @@ -402,7 +600,7 @@ class HiveContext private[hive]( } -private[hive] object HiveContext extends Logging { +private[hive] object HiveContext { /** The version of hive used internally by Spark SQL. */ val hiveExecutionVersion: String = "1.2.1" @@ -468,242 +666,6 @@ private[hive] object HiveContext extends Logging { defaultValue = Some(true), doc = "When set to true, Hive Thrift server executes SQL queries in an asynchronous way.") - /** - * The version of the hive client that will be used to communicate with the metastore. Note that - * this does not necessarily need to be the same version of Hive that is used internally by - * Spark SQL for execution. - */ - private def hiveMetastoreVersion(conf: SQLConf): String = { - conf.getConf(HIVE_METASTORE_VERSION) - } - - /** - * The location of the jars that should be used to instantiate the HiveMetastoreClient. This - * property can be one of three options: - * - a classpath in the standard format for both hive and hadoop. - * - builtin - attempt to discover the jars that were used to load Spark SQL and use those. This - * option is only valid when using the execution version of Hive. - * - maven - download the correct version of hive on demand from maven. - */ - private def hiveMetastoreJars(conf: SQLConf): String = { - conf.getConf(HIVE_METASTORE_JARS) - } - - /** - * A comma separated list of class prefixes that should be loaded using the classloader that - * is shared between Spark SQL and a specific version of Hive. An example of classes that should - * be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need - * to be shared are those that interact with classes that are already shared. For example, - * custom appenders that are used by log4j. - */ - private def hiveMetastoreSharedPrefixes(conf: SQLConf): Seq[String] = { - conf.getConf(HIVE_METASTORE_SHARED_PREFIXES).filterNot(_ == "") - } - - /** - * A comma separated list of class prefixes that should explicitly be reloaded for each version - * of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a - * prefix that typically would be shared (i.e. org.apache.spark.*) - */ - private def hiveMetastoreBarrierPrefixes(conf: SQLConf): Seq[String] = { - conf.getConf(HIVE_METASTORE_BARRIER_PREFIXES).filterNot(_ == "") - } - - /** - * Configurations needed to create a [[HiveClient]]. - */ - private[hive] def hiveClientConfigurations(hiveconf: HiveConf): Map[String, String] = { - // Hive 0.14.0 introduces timeout operations in HiveConf, and changes default values of a bunch - // of time `ConfVar`s by adding time suffixes (`s`, `ms`, and `d` etc.). This breaks backwards- - // compatibility when users are trying to connecting to a Hive metastore of lower version, - // because these options are expected to be integral values in lower versions of Hive. - // - // Here we enumerate all time `ConfVar`s and convert their values to numeric strings according - // to their output time units. - Seq( - ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY -> TimeUnit.SECONDS, - ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME -> TimeUnit.SECONDS, - ConfVars.HMSHANDLERINTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.METASTORE_EVENT_DB_LISTENER_TTL -> TimeUnit.SECONDS, - ConfVars.METASTORE_EVENT_CLEAN_FREQ -> TimeUnit.SECONDS, - ConfVars.METASTORE_EVENT_EXPIRY_DURATION -> TimeUnit.SECONDS, - ConfVars.METASTORE_AGGREGATE_STATS_CACHE_TTL -> TimeUnit.SECONDS, - ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT -> TimeUnit.MILLISECONDS, - ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT -> TimeUnit.MILLISECONDS, - ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_STATS_JDBC_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_STATS_RETRIES_WAIT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES -> TimeUnit.SECONDS, - ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_TXN_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_COMPACTOR_WORKER_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_COMPACTOR_CHECK_INTERVAL -> TimeUnit.SECONDS, - ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.SERVER_READ_SOCKET_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.SPARK_JOB_MONITOR_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT -> TimeUnit.MILLISECONDS - ).map { case (confVar, unit) => - confVar.varname -> hiveconf.getTimeVar(confVar, unit).toString - }.toMap - } - - /** - * Create a [[HiveClient]] used for execution. - * - * Currently this must always be Hive 13 as this is the version of Hive that is packaged - * with Spark SQL. This copy of the client is used for execution related tasks like - * registering temporary functions or ensuring that the ThreadLocal SessionState is - * correctly populated. This copy of Hive is *not* used for storing persistent metadata, - * and only point to a dummy metastore in a temporary directory. - */ - protected[hive] def newClientForExecution( - conf: SparkConf, - hadoopConf: Configuration): HiveClientImpl = { - logInfo(s"Initializing execution hive, version $hiveExecutionVersion") - val loader = new IsolatedClientLoader( - version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), - sparkConf = conf, - execJars = Seq(), - hadoopConf = hadoopConf, - config = newTemporaryConfiguration(useInMemoryDerby = true), - isolationOn = false, - baseClassLoader = Utils.getContextOrSparkClassLoader) - loader.createClient().asInstanceOf[HiveClientImpl] - } - - /** - * Create a [[HiveClient]] used to retrieve metadata from the Hive MetaStore. - * - * 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 = { - val hiveConf = new HiveConf(hadoopConf, classOf[HiveConf]) - val configurations = hiveClientConfigurations(hiveConf) - newClientForMetadata(conf, hiveConf, hadoopConf, configurations) - } - - protected[hive] def newClientForMetadata( - conf: SparkConf, - hiveConf: HiveConf, - hadoopConf: Configuration, - configurations: Map[String, String]): HiveClient = { - val sqlConf = new SQLConf - sqlConf.setConf(SQLContext.getSQLProperties(conf)) - val hiveMetastoreVersion = HiveContext.hiveMetastoreVersion(sqlConf) - val hiveMetastoreJars = HiveContext.hiveMetastoreJars(sqlConf) - val hiveMetastoreSharedPrefixes = HiveContext.hiveMetastoreSharedPrefixes(sqlConf) - val hiveMetastoreBarrierPrefixes = HiveContext.hiveMetastoreBarrierPrefixes(sqlConf) - val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) - - val defaultWarehouseLocation = hiveConf.get("hive.metastore.warehouse.dir") - logInfo("default warehouse location is " + defaultWarehouseLocation) - - // `configure` goes second to override other settings. - val allConfig = hiveConf.asScala.map(e => e.getKey -> e.getValue).toMap ++ configurations - - val isolatedLoader = if (hiveMetastoreJars == "builtin") { - if (hiveExecutionVersion != hiveMetastoreVersion) { - throw new IllegalArgumentException( - "Builtin jars can only be used when hive execution version == hive metastore version. " + - s"Execution: $hiveExecutionVersion != Metastore: $hiveMetastoreVersion. " + - "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + - s"or change ${HIVE_METASTORE_VERSION.key} to $hiveExecutionVersion.") - } - - // We recursively find all jars in the class loader chain, - // starting from the given classLoader. - def allJars(classLoader: ClassLoader): Array[URL] = classLoader match { - case null => Array.empty[URL] - case urlClassLoader: URLClassLoader => - urlClassLoader.getURLs ++ allJars(urlClassLoader.getParent) - case other => allJars(other.getParent) - } - - val classLoader = Utils.getContextOrSparkClassLoader - val jars = allJars(classLoader) - if (jars.length == 0) { - throw new IllegalArgumentException( - "Unable to locate hive jars to connect to metastore. " + - "Please set spark.sql.hive.metastore.jars.") - } - - logInfo( - s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") - new IsolatedClientLoader( - version = metaVersion, - sparkConf = conf, - hadoopConf = hadoopConf, - execJars = jars.toSeq, - config = allConfig, - isolationOn = true, - barrierPrefixes = hiveMetastoreBarrierPrefixes, - sharedPrefixes = hiveMetastoreSharedPrefixes) - } else if (hiveMetastoreJars == "maven") { - // TODO: Support for loading the jars from an already downloaded location. - logInfo( - s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using maven.") - IsolatedClientLoader.forVersion( - hiveMetastoreVersion = hiveMetastoreVersion, - hadoopVersion = VersionInfo.getVersion, - sparkConf = conf, - hadoopConf = hadoopConf, - config = allConfig, - barrierPrefixes = hiveMetastoreBarrierPrefixes, - sharedPrefixes = hiveMetastoreSharedPrefixes) - } else { - // Convert to files and expand any directories. - val jars = - hiveMetastoreJars - .split(File.pathSeparator) - .flatMap { - case path if new File(path).getName == "*" => - val files = new File(path).getParentFile.listFiles() - if (files == null) { - logWarning(s"Hive jar path '$path' does not exist.") - Nil - } else { - files.filter(_.getName.toLowerCase.endsWith(".jar")) - } - case path => - new File(path) :: Nil - } - .map(_.toURI.toURL) - - logInfo( - s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion " + - s"using ${jars.mkString(":")}") - new IsolatedClientLoader( - version = metaVersion, - sparkConf = conf, - hadoopConf = hadoopConf, - execJars = jars.toSeq, - config = allConfig, - isolationOn = true, - barrierPrefixes = hiveMetastoreBarrierPrefixes, - sharedPrefixes = hiveMetastoreSharedPrefixes) - } - isolatedLoader.createClient() - } - /** Constructs a configuration for hive, where the metastore is located in a temp directory. */ def newTemporaryConfiguration(useInMemoryDerby: Boolean): Map[String, String] = { val withInMemoryMode = if (useInMemoryDerby) "memory:" else "" 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 c7066d7363..27e4cfc103 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 @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.DataTypeParser @@ -98,33 +98,27 @@ private[hive] object HiveSerDe { } -/** - * Legacy catalog for interacting with the Hive metastore. - * - * This is still used for things like creating data source tables, but in the future will be - * cleaned up to integrate more nicely with [[HiveCatalog]]. - */ +// TODO: replace this with o.a.s.sql.hive.HiveCatalog once we merge SQLContext and HiveContext private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveContext) - extends Logging { + extends Catalog with Logging { val conf = hive.conf + /** Usages should lock on `this`. */ + protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) + /** A fully qualified identifier for a table (i.e., database.tableName) */ case class QualifiedTableName(database: String, name: String) - private def getCurrentDatabase: String = { - hive.sessionState.catalog.getCurrentDatabase - } - - def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = { + private def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = { QualifiedTableName( - tableIdent.database.getOrElse(getCurrentDatabase).toLowerCase, + tableIdent.database.getOrElse(client.currentDatabase).toLowerCase, tableIdent.table.toLowerCase) } private def getQualifiedTableName(t: CatalogTable): QualifiedTableName = { QualifiedTableName( - t.name.database.getOrElse(getCurrentDatabase).toLowerCase, + t.name.database.getOrElse(client.currentDatabase).toLowerCase, t.name.table.toLowerCase) } @@ -200,7 +194,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte CacheBuilder.newBuilder().maximumSize(1000).build(cacheLoader) } - def refreshTable(tableIdent: TableIdentifier): Unit = { + override def refreshTable(tableIdent: TableIdentifier): Unit = { // refreshTable does not eagerly reload the cache. It just invalidate the cache. // Next time when we use the table, it will be populated in the cache. // Since we also cache ParquetRelations converted from Hive Parquet tables and @@ -414,7 +408,12 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte new Path(new Path(client.getDatabase(dbName).locationUri), tblName).toString } - def lookupRelation( + override def tableExists(tableIdent: TableIdentifier): Boolean = { + val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent) + client.getTableOption(dbName, tblName).isDefined + } + + override def lookupRelation( tableIdent: TableIdentifier, alias: Option[String]): LogicalPlan = { val qualifiedTableName = getQualifiedTableName(tableIdent) @@ -556,6 +555,12 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte result.copy(expectedOutputAttributes = Some(metastoreRelation.output)) } + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + val db = databaseName.getOrElse(client.currentDatabase) + + client.listTables(db).map(tableName => (tableName, false)) + } + /** * When scanning or writing to non-partitioned Metastore Parquet tables, convert them to Parquet * data source relations for better performance. @@ -711,6 +716,27 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } } + /** + * UNIMPLEMENTED: It needs to be decided how we will persist in-memory tables to the metastore. + * For now, if this functionality is desired mix in the in-memory [[OverrideCatalog]]. + */ + override def registerTable(tableIdent: TableIdentifier, plan: LogicalPlan): Unit = { + throw new UnsupportedOperationException + } + + /** + * UNIMPLEMENTED: It needs to be decided how we will persist in-memory tables to the metastore. + * For now, if this functionality is desired mix in the in-memory [[OverrideCatalog]]. + */ + override def unregisterTable(tableIdent: TableIdentifier): Unit = { + throw new UnsupportedOperationException + } + + override def unregisterAllTables(): Unit = {} + + override def setCurrentDatabase(databaseName: String): Unit = { + client.setCurrentDatabase(databaseName) + } } /** 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 deleted file mode 100644 index aa44cba4b5..0000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.SessionCatalog -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.BucketSpec -import org.apache.spark.sql.hive.client.HiveClient -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType - - -class HiveSessionCatalog( - externalCatalog: HiveCatalog, - client: HiveClient, - context: HiveContext, - conf: SQLConf) - extends SessionCatalog(externalCatalog, conf) { - - override def setCurrentDatabase(db: String): Unit = { - super.setCurrentDatabase(db) - client.setCurrentDatabase(db) - } - - override def lookupRelation(name: TableIdentifier, alias: Option[String]): LogicalPlan = { - val table = formatTableName(name.table) - if (name.database.isDefined || !tempTables.containsKey(table)) { - val newName = name.copy(table = table) - metastoreCatalog.lookupRelation(newName, alias) - } else { - val relation = tempTables.get(table) - val tableWithQualifiers = SubqueryAlias(table, relation) - // If an alias was specified by the lookup, wrap the plan in a subquery so that - // attributes are properly qualified with this alias. - alias.map(a => SubqueryAlias(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) - } - } - - // ---------------------------------------------------------------- - // | Methods and fields for interacting with HiveMetastoreCatalog | - // ---------------------------------------------------------------- - - // Catalog for handling data source tables. TODO: This really doesn't belong here since it is - // 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) - - val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions - val CreateTables: Rule[LogicalPlan] = metastoreCatalog.CreateTables - val PreInsertionCasts: Rule[LogicalPlan] = metastoreCatalog.PreInsertionCasts - - override def refreshTable(name: TableIdentifier): Unit = { - metastoreCatalog.refreshTable(name) - } - - def invalidateTable(name: TableIdentifier): Unit = { - metastoreCatalog.invalidateTable(name) - } - - def invalidateCache(): Unit = { - metastoreCatalog.cachedDataSourceTables.invalidateAll() - } - - def createDataSourceTable( - name: TableIdentifier, - userSpecifiedSchema: Option[StructType], - partitionColumns: Array[String], - bucketSpec: Option[BucketSpec], - provider: String, - options: Map[String, String], - isExternal: Boolean): Unit = { - metastoreCatalog.createDataSourceTable( - name, userSpecifiedSchema, partitionColumns, bucketSpec, provider, options, isExternal) - } - - def hiveDefaultTableFilePath(name: TableIdentifier): String = { - metastoreCatalog.hiveDefaultTableFilePath(name) - } - - // For testing only - private[hive] def getCachedDataSourceTable(table: TableIdentifier): LogicalPlan = { - val key = metastoreCatalog.getQualifiedTableName(table) - metastoreCatalog.cachedDataSourceTables.getIfPresent(key) - } - -} 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 caa7f296ed..d9cd96d66f 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,7 +18,7 @@ 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, FunctionRegistry, OverrideCatalog} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.execution.{python, SparkPlanner} import org.apache.spark.sql.execution.datasources._ @@ -35,11 +35,9 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) } /** - * Internal catalog for managing table and database states. + * A metadata catalog that points to the Hive metastore. */ - override lazy val catalog = { - new HiveSessionCatalog(ctx.hiveCatalog, ctx.metadataHive, ctx, conf) - } + override lazy val catalog = new HiveMetastoreCatalog(ctx.metadataHive, ctx) with OverrideCatalog /** * Internal catalog for managing functions registered by the user. @@ -63,7 +61,7 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) DataSourceAnalysis :: (if (conf.runSQLOnFile) new ResolveDataSource(ctx) :: Nil else Nil) - override val extendedCheckRules = Seq(PreWriteCheck(conf, catalog)) + override val extendedCheckRules = Seq(PreWriteCheck(catalog)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index f4d30358ca..d214e5288e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -53,6 +53,9 @@ private[hive] trait HiveClient { /** Returns the names of tables in the given database that matches the given pattern. */ def listTables(dbName: String, pattern: String): Seq[String] + /** Returns the name of the active database. */ + def currentDatabase: String + /** Sets the name of current database. */ def setCurrentDatabase(databaseName: String): Unit 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 e4e15d13df..928408c52b 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 @@ -241,6 +241,10 @@ private[hive] class HiveClientImpl( state.err = stream } + override def currentDatabase: String = withHiveState { + state.getCurrentDatabase + } + override def setCurrentDatabase(databaseName: String): Unit = withHiveState { if (getDatabaseOption(databaseName).isDefined) { state.setCurrentDatabase(databaseName) 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 5a61eef0f2..391e2975d0 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 @@ -69,10 +69,10 @@ case class CreateTableAsSelect( withFormat } - hiveContext.sessionState.catalog.createTable(withSchema, ignoreIfExists = false) + hiveContext.sessionState.catalog.client.createTable(withSchema, ignoreIfExists = false) // Get the Metastore Relation - hiveContext.sessionState.catalog.lookupRelation(tableIdentifier) match { + hiveContext.sessionState.catalog.lookupRelation(tableIdentifier, None) match { case r: MetastoreRelation => r } } 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 9ff520da1d..8a1cf2caaa 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.catalog.client.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.catalog.client.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 430fa4616f..4ffd868242 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.hiveconf) - @transient private lazy val client = sc.metadataHive + @transient private lazy val catalog = sc.sessionState.catalog def output: Seq[Attribute] = Seq.empty @@ -186,8 +186,8 @@ case class InsertIntoHiveTable( // TODO: Correctly set isSkewedStoreAsSubdir. val isSkewedStoreAsSubdir = false if (numDynamicPartitions > 0) { - client.synchronized { - client.loadDynamicPartitions( + catalog.synchronized { + catalog.client.loadDynamicPartitions( outputPath.toString, qualifiedTableName, orderedPartitionSpec, @@ -202,12 +202,12 @@ case class InsertIntoHiveTable( // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries // scalastyle:on val oldPart = - client.getPartitionOption( - client.getTable(table.databaseName, table.tableName), + catalog.client.getPartitionOption( + catalog.client.getTable(table.databaseName, table.tableName), partitionSpec) if (oldPart.isEmpty || !ifNotExists) { - client.loadPartition( + catalog.client.loadPartition( outputPath.toString, qualifiedTableName, orderedPartitionSpec, @@ -218,7 +218,7 @@ case class InsertIntoHiveTable( } } } else { - client.loadTable( + catalog.client.loadTable( outputPath.toString, // TODO: URI qualifiedTableName, overwrite, 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 cd26a68f35..226b8e1796 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 @@ -71,8 +71,7 @@ case class DropTable( } hiveContext.invalidateTable(tableName) hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") - hiveContext.sessionState.catalog.dropTable( - TableIdentifier(tableName), ignoreIfNotExists = true) + hiveContext.sessionState.catalog.unregisterTable(TableIdentifier(tableName)) Seq.empty[Row] } } @@ -143,8 +142,7 @@ case class CreateMetastoreDataSource( val optionsWithPath = if (!options.contains("path") && managedIfNoPath) { isExternal = false - options + ("path" -> - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) + options + ("path" -> hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } @@ -202,8 +200,7 @@ case class CreateMetastoreDataSourceAsSelect( val optionsWithPath = if (!options.contains("path")) { isExternal = false - options + ("path" -> - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) + options + ("path" -> hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } 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 1155903037..19c05f9cb0 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 @@ -24,8 +24,6 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.implicitConversions -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.processors._ @@ -37,11 +35,9 @@ 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.HiveClientImpl import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{ShutdownHookManager, Utils} @@ -75,77 +71,10 @@ trait TestHiveSingleton { * 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]( - sc: SparkContext, - cacheManager: CacheManager, - listener: SQLListener, - executionHive: HiveClientImpl, - metadataHive: HiveClient, - isRootContext: Boolean, - hiveCatalog: HiveCatalog, - val warehousePath: File, - val scratchDirPath: File) - extends HiveContext( - sc, - cacheManager, - listener, - executionHive, - metadataHive, - isRootContext, - hiveCatalog) { self => +class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { + 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. - - private def this( - sc: SparkContext, - executionHive: HiveClientImpl, - metadataHive: HiveClient, - warehousePath: File, - scratchDirPath: File) { - this( - sc, - new CacheManager, - SQLContext.createListenerAndUI(sc), - executionHive, - metadataHive, - true, - new HiveCatalog(metadataHive), - warehousePath, - scratchDirPath) - } - - private def this(sc: SparkContext, warehousePath: File, scratchDirPath: File) { - this( - sc, - HiveContext.newClientForExecution(sc.conf, sc.hadoopConfiguration), - TestHiveContext.newClientForMetadata( - sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath), - warehousePath, - scratchDirPath) - } - - def this(sc: SparkContext) { - this( - sc, - Utils.createTempDir(namePrefix = "warehouse"), - TestHiveContext.makeScratchDir()) - } - - 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) - } + import HiveContext._ // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. @@ -154,6 +83,26 @@ class TestHiveContext private[hive]( hiveconf.set("hive.plan.serialization.format", "javaXML") + lazy val warehousePath = Utils.createTempDir(namePrefix = "warehouse-") + + lazy val scratchDirPath = { + val dir = Utils.createTempDir(namePrefix = "scratch-") + dir.delete() + dir + } + + private lazy val temporaryConfig = newTemporaryConfiguration(useInMemoryDerby = false) + + /** Sets up the system initially or after a RESET command */ + protected override def configure(): Map[String, String] = { + super.configure() ++ temporaryConfig ++ Map( + ConfVars.METASTOREWAREHOUSE.varname -> warehousePath.toURI.toString, + ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN.varname -> "true", + ConfVars.SCRATCHDIR.varname -> scratchDirPath.toURI.toString, + ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY.varname -> "1" + ) + } + val testTempDir = Utils.createTempDir() // For some hive test case which contain ${system:test.tmp.dir} @@ -478,9 +427,9 @@ class TestHiveContext private[hive]( cacheManager.clearCache() loadedTables.clear() - sessionState.catalog.clearTempTables() - sessionState.catalog.invalidateCache() - metadataHive.reset() + sessionState.catalog.cachedDataSourceTables.invalidateAll() + sessionState.catalog.client.reset() + sessionState.catalog.unregisterAllTables() FunctionRegistry.getFunctionNames.asScala.filterNot(originalUDFs.contains(_)). foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) } @@ -499,8 +448,13 @@ class TestHiveContext private[hive]( // Lots of tests fail if we do not change the partition whitelist from the default. runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") + configure().foreach { + case (k, v) => + metadataHive.runSqlHive(s"SET $k=$v") + } defaultOverrides() - sessionState.catalog.setCurrentDatabase("default") + + runSqlHive("USE default") } catch { case e: Exception => logError("FATAL ERROR: Failed to reset TestDB state.", e) @@ -536,43 +490,4 @@ private[hive] object TestHiveContext { // Fewer shuffle partitions to speed up testing. SQLConf.SHUFFLE_PARTITIONS.key -> "5" ) - - /** - * Create a [[HiveClient]] used to retrieve metadata from the Hive MetaStore. - */ - private def newClientForMetadata( - conf: SparkConf, - hadoopConf: Configuration, - warehousePath: File, - scratchDirPath: File): HiveClient = { - val hiveConf = new HiveConf(hadoopConf, classOf[HiveConf]) - HiveContext.newClientForMetadata( - conf, - hiveConf, - hadoopConf, - hiveClientConfigurations(hiveConf, warehousePath, scratchDirPath)) - } - - /** - * Configurations needed to create a [[HiveClient]]. - */ - private def hiveClientConfigurations( - hiveconf: HiveConf, - warehousePath: File, - scratchDirPath: File): Map[String, String] = { - HiveContext.hiveClientConfigurations(hiveconf) ++ - HiveContext.newTemporaryConfiguration(useInMemoryDerby = false) ++ Map( - ConfVars.METASTOREWAREHOUSE.varname -> warehousePath.toURI.toString, - ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN.varname -> "true", - ConfVars.SCRATCHDIR.varname -> scratchDirPath.toURI.toString, - ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY.varname -> "1" - ) - } - - private def makeScratchDir(): File = { - val scratchDir = Utils.createTempDir(namePrefix = "scratch") - scratchDir.delete() - scratchDir - } - } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index 2fc38e2b2d..bd14a243ea 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -70,9 +70,8 @@ public class JavaMetastoreDataSourcesSuite { if (path.exists()) { path.delete(); } - hiveManagedPath = new Path( - sqlContext.sessionState().catalog().hiveDefaultTableFilePath( - new TableIdentifier("javaSavedTable"))); + hiveManagedPath = new Path(sqlContext.sessionState().catalog().hiveDefaultTableFilePath( + new TableIdentifier("javaSavedTable"))); fs = hiveManagedPath.getFileSystem(sc.hadoopConfiguration()); if (fs.exists(hiveManagedPath)){ fs.delete(hiveManagedPath, true); 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 deleted file mode 100644 index fa0c4d92cd..0000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* -* 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.SparkFunSuite -import org.apache.spark.sql.hive.test.TestHive - - -class HiveContextSuite extends SparkFunSuite { - - // TODO: investigate; this passes locally but fails on Jenkins for some reason. - ignore("HiveContext can access `spark.sql.*` configs") { - // Avoid creating another SparkContext in the same JVM - val sc = TestHive.sparkContext - require(sc.conf.get("spark.sql.hive.metastore.barrierPrefixes") == - "org.apache.spark.sql.hive.execution.PairSerDe") - assert(TestHive.getConf("spark.sql.hive.metastore.barrierPrefixes") == - "org.apache.spark.sql.hive.execution.PairSerDe") - assert(TestHive.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/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 42cbfee10e..ce7b08ab72 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 @@ -21,7 +21,6 @@ import java.io.File import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{QueryTest, Row, SaveMode} -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -84,7 +83,7 @@ class DataSourceWithHiveMetastoreCatalogSuite .saveAsTable("t") } - val hiveTable = sessionState.catalog.getTable(TableIdentifier("t", Some("default"))) + val hiveTable = sessionState.catalog.client.getTable("default", "t") assert(hiveTable.storage.inputFormat === Some(inputFormat)) assert(hiveTable.storage.outputFormat === Some(outputFormat)) assert(hiveTable.storage.serde === Some(serde)) @@ -115,8 +114,7 @@ class DataSourceWithHiveMetastoreCatalogSuite .saveAsTable("t") } - val hiveTable = - sessionState.catalog.getTable(TableIdentifier("t", Some("default"))) + val hiveTable = sessionState.catalog.client.getTable("default", "t") assert(hiveTable.storage.inputFormat === Some(inputFormat)) assert(hiveTable.storage.outputFormat === Some(outputFormat)) assert(hiveTable.storage.serde === Some(serde)) @@ -146,8 +144,7 @@ class DataSourceWithHiveMetastoreCatalogSuite |AS SELECT 1 AS d1, "val_1" AS d2 """.stripMargin) - val hiveTable = - sessionState.catalog.getTable(TableIdentifier("t", Some("default"))) + val hiveTable = sessionState.catalog.client.getTable("default", "t") assert(hiveTable.storage.inputFormat === Some(inputFormat)) assert(hiveTable.storage.outputFormat === Some(outputFormat)) assert(hiveTable.storage.serde === Some(serde)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala index c3b24623d1..0a31ac64a2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala @@ -32,16 +32,14 @@ class ListTablesSuite extends QueryTest with TestHiveSingleton with BeforeAndAft override def beforeAll(): Unit = { // The catalog in HiveContext is a case insensitive one. - sessionState.catalog.createTempTable( - "ListTablesSuiteTable", df.logicalPlan, ignoreIfExists = true) + sessionState.catalog.registerTable(TableIdentifier("ListTablesSuiteTable"), df.logicalPlan) sql("CREATE TABLE HiveListTablesSuiteTable (key int, value string)") sql("CREATE DATABASE IF NOT EXISTS ListTablesSuiteDB") sql("CREATE TABLE ListTablesSuiteDB.HiveInDBListTablesSuiteTable (key int, value string)") } override def afterAll(): Unit = { - sessionState.catalog.dropTable( - TableIdentifier("ListTablesSuiteTable"), ignoreIfNotExists = true) + sessionState.catalog.unregisterTable(TableIdentifier("ListTablesSuiteTable")) sql("DROP TABLE IF EXISTS HiveListTablesSuiteTable") sql("DROP TABLE IF EXISTS ListTablesSuiteDB.HiveInDBListTablesSuiteTable") sql("DROP DATABASE IF EXISTS ListTablesSuiteDB") 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 7d2a4eb1de..3f3d0692b7 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 @@ -693,13 +693,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("SPARK-6024 wide schema support") { withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD.key -> "4000") { withTable("wide_schema") { - withTempDir { tempDir => + withTempDir( tempDir => { // We will need 80 splits for this schema if the threshold is 4000. val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType, true))) // Manually create a metastore data source table. sessionState.catalog.createDataSourceTable( - name = TableIdentifier("wide_schema"), + tableIdent = TableIdentifier("wide_schema"), userSpecifiedSchema = Some(schema), partitionColumns = Array.empty[String], bucketSpec = None, @@ -711,7 +711,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val actualSchema = table("wide_schema").schema assert(schema === actualSchema) - } + }) } } } @@ -737,7 +737,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv "spark.sql.sources.schema" -> schema.json, "EXTERNAL" -> "FALSE")) - hiveCatalog.createTable("default", hiveTable, ignoreIfExists = false) + sessionState.catalog.client.createTable(hiveTable, ignoreIfExists = false) invalidateTable(tableName) val actualSchema = table(tableName).schema @@ -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) - val metastoreTable = hiveCatalog.getTable("default", tableName) + val metastoreTable = sessionState.catalog.client.getTable("default", tableName) val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val numPartCols = metastoreTable.properties("spark.sql.sources.schema.numPartCols").toInt @@ -787,7 +787,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .sortBy("c") .saveAsTable(tableName) invalidateTable(tableName) - val metastoreTable = hiveCatalog.getTable("default", tableName) + val metastoreTable = sessionState.catalog.client.getTable("default", tableName) val expectedBucketByColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val expectedSortByColumns = StructType(df.schema("c") :: Nil) @@ -903,11 +903,11 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("skip hive metadata on table creation") { - withTempDir { tempPath => + withTempDir(tempPath => { val schema = StructType((1 to 5).map(i => StructField(s"c_$i", StringType))) sessionState.catalog.createDataSourceTable( - name = TableIdentifier("not_skip_hive_metadata"), + tableIdent = TableIdentifier("not_skip_hive_metadata"), userSpecifiedSchema = Some(schema), partitionColumns = Array.empty[String], bucketSpec = None, @@ -917,11 +917,11 @@ 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(sessionState.catalog.client.getTable("default", "not_skip_hive_metadata").schema .forall(column => HiveMetastoreTypes.toDataType(column.dataType) == StringType)) sessionState.catalog.createDataSourceTable( - name = TableIdentifier("skip_hive_metadata"), + tableIdent = TableIdentifier("skip_hive_metadata"), userSpecifiedSchema = Some(schema), partitionColumns = Array.empty[String], bucketSpec = None, @@ -929,11 +929,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv options = Map("path" -> tempPath.getCanonicalPath, "skipHiveMetadata" -> "true"), isExternal = false) - // 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) - }) - } + // 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(sessionState.catalog.client.getTable("default", "skip_hive_metadata").schema + .forall(column => HiveMetastoreTypes.toDataType(column.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 3be2269d3f..d275190744 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.sessionState.catalog.client.getTable(dbName, tableName) + val expectedPath = + hiveContext.sessionState.catalog.client.getDatabase(dbName).locationUri + "/" + tableName assert(metastoreTable.storage.serdeProperties("path") === expectedPath) } 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 ae026ed496..151aacbdd1 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 @@ -121,8 +121,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { intercept[UnsupportedOperationException] { hiveContext.analyze("tempTable") } - hiveContext.sessionState.catalog.dropTable( - TableIdentifier("tempTable"), ignoreIfNotExists = true) + hiveContext.sessionState.catalog.unregisterTable(TableIdentifier("tempTable")) } test("estimates the size of a test MetastoreRelation") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index d59bca4c7e..295069228f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -171,6 +171,10 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.listTables("default") === Seq("src")) } + test(s"$version: currentDatabase") { + assert(client.currentDatabase === "default") + } + test(s"$version: getDatabase") { client.getDatabase("default") } 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 197a123905..5fe85eaef2 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 @@ -49,7 +49,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ override def beforeAll() { - super.beforeAll() TestHive.cacheTables = true // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) @@ -58,14 +57,11 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } override def afterAll() { - try { - TestHive.cacheTables = false - TimeZone.setDefault(originalTimeZone) - Locale.setDefault(originalLocale) - sql("DROP TEMPORARY FUNCTION udtf_count2") - } finally { - super.afterAll() - } + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + sql("DROP TEMPORARY FUNCTION udtf_count2") + super.afterAll() } test("SPARK-4908: concurrent hive native commands") { @@ -1213,7 +1209,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("USE hive_test_db") assert("hive_test_db" == sql("select current_database()").first().getString(0)) - intercept[AnalysisException] { + intercept[NoSuchDatabaseException] { sql("USE not_existing_db") } 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 6199253d34..bc8896d4bd 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 @@ -1325,7 +1325,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { .format("parquet") .save(path) - // We don't support creating a temporary table while specifying a database val message = intercept[AnalysisException] { sqlContext.sql( s""" @@ -1336,8 +1335,9 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |) """.stripMargin) }.getMessage + assert(message.contains("Specifying database name or other qualifiers are not allowed")) - // If you use backticks to quote the name then it's OK. + // If you use backticks to quote the name of a temporary table having dot in it. sqlContext.sql( s""" |CREATE TEMPORARY TABLE `db.t` diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 92f424bac7..cc412241fb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -222,7 +222,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { sql("INSERT INTO TABLE t SELECT * FROM tmp") checkAnswer(table("t"), (data ++ data).map(Row.fromTuple)) } - sessionState.catalog.dropTable(TableIdentifier("tmp"), ignoreIfNotExists = true) + sessionState.catalog.unregisterTable(TableIdentifier("tmp")) } test("overwriting") { @@ -232,7 +232,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(table("t"), data.map(Row.fromTuple)) } - sessionState.catalog.dropTable(TableIdentifier("tmp"), ignoreIfNotExists = true) + sessionState.catalog.unregisterTable(TableIdentifier("tmp")) } test("self-join") { 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 07fe0ccd87..bb53179c3c 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 @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.DataSourceScan import org.apache.spark.sql.execution.command.ExecutedCommand import org.apache.spark.sql.execution.datasources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} @@ -426,9 +425,10 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { } test("Caching converted data source Parquet Relations") { - def checkCached(tableIdentifier: TableIdentifier): Unit = { + val _catalog = sessionState.catalog + def checkCached(tableIdentifier: _catalog.QualifiedTableName): Unit = { // Converted test_parquet should be cached. - sessionState.catalog.getCachedDataSourceTable(tableIdentifier) match { + sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => fail("Converted test_parquet should be cached in the cache.") case logical @ LogicalRelation(parquetRelation: HadoopFsRelation, _, _) => // OK case other => @@ -453,17 +453,17 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' """.stripMargin) - var tableIdentifier = TableIdentifier("test_insert_parquet", Some("default")) + var tableIdentifier = _catalog.QualifiedTableName("default", "test_insert_parquet") // First, make sure the converted test_parquet is not cached. - assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) + assert(sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) // Table lookup will make the table cached. table("test_insert_parquet") 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") - assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) + assert(sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) sql( """ |INSERT INTO TABLE test_insert_parquet @@ -476,7 +476,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql("select a, b from jt").collect()) // Invalidate the cache. invalidateTable("test_insert_parquet") - assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) + assert(sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) // Create a partitioned table. sql( @@ -493,8 +493,8 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' """.stripMargin) - tableIdentifier = TableIdentifier("test_parquet_partitioned_cache_test", Some("default")) - assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) + tableIdentifier = _catalog.QualifiedTableName("default", "test_parquet_partitioned_cache_test") + assert(sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) sql( """ |INSERT INTO TABLE test_parquet_partitioned_cache_test @@ -503,14 +503,14 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { """.stripMargin) // Right now, insert into a partitioned Parquet is not supported in data source Parquet. // So, we expect it is not cached. - assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) + assert(sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) sql( """ |INSERT INTO TABLE test_parquet_partitioned_cache_test |PARTITION (`date`='2015-04-02') |select a, b from jt """.stripMargin) - assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) + assert(sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) // Make sure we can cache the partitioned table. table("test_parquet_partitioned_cache_test") @@ -526,7 +526,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { """.stripMargin).collect()) invalidateTable("test_parquet_partitioned_cache_test") - assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) + assert(sessionState.catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) dropTables("test_insert_parquet", "test_parquet_partitioned_cache_test") }