From 98c8a3b922168b843fe648664fc0e8ac2f930472 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 10 Mar 2016 16:35:35 -0800 Subject: [PATCH 01/20] Merge in @yhuai's changes --- .../sql/catalyst/catalog/SessionCatalog.scala | 232 ++++++++++++++++++ 1 file changed, 232 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala 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 new file mode 100644 index 000000000000..08b21df2396f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -0,0 +1,232 @@ +/* + * 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.catalog + +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + + +/** + * An internal catalog that is used by a Spark Session. This internal catalog serves as a + * 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. + */ +abstract class SessionCatalog(catalog: ExternalCatalog) { + import ExternalCatalog._ + + private[this] val tempTables = new ConcurrentHashMap[String, LogicalPlan] + + private[this] val tempFunctions = new ConcurrentHashMap[String, CatalogFunction] + + // -------------------------------------------------------------------------- + // Databases + // All methods in this category interact directly with the underlying catalog. + // -------------------------------------------------------------------------- + + def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit + + def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit + + def alterDatabase(dbDefinition: CatalogDatabase): Unit + + def getDatabase(db: String): CatalogDatabase + + def databaseExists(db: String): Boolean + + def listDatabases(): Seq[String] + + def listDatabases(pattern: String): Seq[String] + + // -------------------------------------------------------------------------- + // Tables + // -------------------------------------------------------------------------- + + // -------------------------------------------------------------------------- + // Tables: Methods for metastore tables. + // Methods in this category are only used for metastore tables, which store + // metadata in the underlying catalog. + // -------------------------------------------------------------------------- + + def createTable(db: String, tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit + + /** + * Alters a table whose name matches the one specified in `tableDefinition`, + * assuming the table exists. + * + * Note: If the underlying implementation does not support altering a certain field, + * this becomes a no-op. + */ + def alterTable(db: String, tableDefinition: CatalogTable): Unit + + /** + * Retrieves the metadata of a table called `table` in the database `db`. + */ + def getTable(db: String, table: String): CatalogTable + + // -------------------------------------------------------------------------- + // Tables: Methods for metastore tables or temp tables. + // -------------------------------------------------------------------------- + + /** + * Creates a temporary table. If there is already a temporary table having the same name, + * the table definition of that table will be updated to the new definition. + */ + // TODO: Should we automatically overwrite the existing temp table? + // Postgres and Hive will complain if a temp table is already defined. + def createTempTable(tableIdent: TableIdentifier, tableDefinition: LogicalPlan): Unit + + def renameTable( + specifiedDB: Option[String], + currentDB: String, + oldName: String, + newName: String): Unit + + /** + * Drops a table. If a database name is not provided, this method will drop the table with + * the given name from the temporary table name space as well as the table + * in the current database. If a database name is provided, this method only drops the table + * with the given name from the given database. + */ + // TODO: When a temp table and a table in the current db have the same name, should we + // only drop the temp table when a database is not provided (Postgresql's semantic)? + def dropTable( + tableIdent: TableIdentifier, + currentDB: String, + ignoreIfNotExists: Boolean): Unit + + /** + * Returns a [[LogicalPlan]] representing the requested table. This method is used + * when we need to create a query plan for a given table. + * + * This method is different from `getTable`, which only returns the metadata of the table + * in the form of [[CatalogTable]]. The [[LogicalPlan]] returned by this method contains + * the metadata of the table in the form of [[CatalogTable]]. + */ + def lookupRelation(tableIdent: TableIdentifier, alias: Option[String] = None): LogicalPlan + + def listTables(specifiedDB: Option[String], currentDB: String): Seq[String] + + def listTables(specifiedDB: Option[String], currentDB: String, pattern: String): Seq[String] + + // -------------------------------------------------------------------------- + // Partitions + // All methods in this category interact directly with the underlying catalog. + // -------------------------------------------------------------------------- + // TODO: We need to figure out how these methods interact with our data source tables. + // For data source tables, we do not store values of partitioning columns in the metastore. + // For now, partition values of a data source table will be automatically discovered + // when we load the table. + + def createPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit + + def dropPartitions( + db: String, + table: String, + parts: Seq[TablePartitionSpec], + ignoreIfNotExists: Boolean): Unit + + /** + * Override the specs of one or many existing table partitions, assuming they exist. + * This assumes index i of `specs` corresponds to index i of `newSpecs`. + */ + def renamePartitions( + db: String, + table: String, + specs: Seq[TablePartitionSpec], + newSpecs: Seq[TablePartitionSpec]): Unit + + /** + * Alter one or many table partitions whose specs that match those specified in `parts`, + * assuming the partitions exist. + * + * Note: If the underlying implementation does not support altering a certain field, + * this becomes a no-op. + */ + def alterPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition]): Unit + + def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition + + def listPartitions(db: String, table: String): Seq[CatalogTablePartition] + + // -------------------------------------------------------------------------- + // Functions + // -------------------------------------------------------------------------- + + // -------------------------------------------------------------------------- + // Functions: Methods for metastore functions (permanent UDFs). + // -------------------------------------------------------------------------- + + def createFunction(db: String, funcDefinition: CatalogFunction): Unit + + /** + * Drops a permanent function with the given name from the given database. + */ + def dropFunction(db: String, funcName: String): Unit + + // -------------------------------------------------------------------------- + // Functions: Methods for metastore functions (permanent UDFs) or temp functions. + // -------------------------------------------------------------------------- + + def createTempFunction(funcDefinition: CatalogFunction): Unit + + /** + * Drops a temporary function with the given name. + */ + // TODO: The reason that we distinguish dropFunction and dropTempFunction is that + // Hive has DROP FUNCTION and DROP TEMPORARY FUNCTION. We may want to consolidate + // dropFunction and dropTempFunction. + def dropTempFunction(funcName: String): Unit + + def renameFunction( + specifiedDB: Option[String], + currentDB: String, + oldName: String, + newName: String): Unit + + /** + * Alter a function whose name that matches the one specified in `funcDefinition`, + * assuming the function exists. + * + * Note: If the underlying implementation does not support altering a certain field, + * this becomes a no-op. + */ + def alterFunction( + specifiedDB: Option[String], + currentDB: String, + funcDefinition: CatalogFunction): Unit + + def getFunction( + specifiedDB: Option[String], + currentDB: String, + funcName: String): CatalogFunction + + def listFunctions( + specifiedDB: Option[String], + currentDB: String, + pattern: String): Seq[String] + +} From 1d12578708da845fe309d3aae1dcdadfee1dee89 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 14 Mar 2016 16:27:11 -0700 Subject: [PATCH 02/20] Clean up table method signatures + add comments --- .../sql/catalyst/catalog/SessionCatalog.scala | 105 ++++++++++-------- .../sql/catalyst/catalog/interface.scala | 1 + 2 files changed, 61 insertions(+), 45 deletions(-) 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 08b21df2396f..a7cb7ead0c7f 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 @@ -35,10 +35,11 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { private[this] val tempFunctions = new ConcurrentHashMap[String, CatalogFunction] - // -------------------------------------------------------------------------- + // ---------------------------------------------------------------------------- // Databases + // ---------------------------------------------------------------------------- // All methods in this category interact directly with the underlying catalog. - // -------------------------------------------------------------------------- + // ---------------------------------------------------------------------------- def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit @@ -54,76 +55,90 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { def listDatabases(pattern: String): Seq[String] - // -------------------------------------------------------------------------- + // ---------------------------------------------------------------------------- // Tables - // -------------------------------------------------------------------------- - - // -------------------------------------------------------------------------- - // Tables: Methods for metastore tables. - // Methods in this category are only used for metastore tables, which store - // metadata in the underlying catalog. - // -------------------------------------------------------------------------- + // ---------------------------------------------------------------------------- + // There are two kinds of tables, temporary tables and metastore tables. + // Temporary tables are isolated across sessions and do not belong to any + // particular database. Metastore tables can be used across multiple + // sessions as their metadata is persisted in the underlying catalog. + // ---------------------------------------------------------------------------- - def createTable(db: String, tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit + /** + * Create a metastore table in the database specified in `tableDefinition`. + * If no such table is specified, create it in the current database. + */ + def createTable( + currentDb: String, + tableDefinition: CatalogTable, + ignoreIfExists: Boolean): Unit /** - * Alters a table whose name matches the one specified in `tableDefinition`, - * assuming the table exists. + * Alter the metadata of an existing metastore table identified by `tableDefinition`. * * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterTable(db: String, tableDefinition: CatalogTable): Unit + def alterTable(tableDefinition: CatalogTable): Unit /** - * Retrieves the metadata of a table called `table` in the database `db`. + * Retrieve the metadata of an existing metastore table. */ - def getTable(db: String, table: String): CatalogTable - - // -------------------------------------------------------------------------- - // Tables: Methods for metastore tables or temp tables. - // -------------------------------------------------------------------------- + def getTableMetadata(name: TableIdentifier): CatalogTable /** - * Creates a temporary table. If there is already a temporary table having the same name, - * the table definition of that table will be updated to the new definition. + * Create a temporary table. + * If a temporary table with the same name already exists, this throws an exception. */ - // TODO: Should we automatically overwrite the existing temp table? - // Postgres and Hive will complain if a temp table is already defined. - def createTempTable(tableIdent: TableIdentifier, tableDefinition: LogicalPlan): Unit + def createTempTable(name: String, tableDefinition: LogicalPlan): Unit + /** + * Rename a table. + * + * If a database is specified in `oldName`, this will rename the table in that database. + * If no database is specified, this will first attempt to rename a temporary table with + * the same name, then, if that does not exist, rename the table in the current database. + * + * This assumes the database specified in `oldName` matches the one specified in `newName`. + */ def renameTable( - specifiedDB: Option[String], - currentDB: String, - oldName: String, - newName: String): Unit + currentDb: String, + oldName: TableIdentifier, + newName: TableIdentifier): Unit /** - * Drops a table. If a database name is not provided, this method will drop the table with - * the given name from the temporary table name space as well as the table - * in the current database. If a database name is provided, this method only drops the table - * with the given name from the given database. + * Drop a table. + * + * If a database is specified in `name`, this will drop the table from that database. + * If no database is specified, this will first attempt to drop a temporary table with + * the same name, then, if that does not exist, drop the table from the current database. */ - // TODO: When a temp table and a table in the current db have the same name, should we - // only drop the temp table when a database is not provided (Postgresql's semantic)? def dropTable( - tableIdent: TableIdentifier, - currentDB: String, + currentDb: String, + name: TableIdentifier, ignoreIfNotExists: Boolean): Unit /** - * Returns a [[LogicalPlan]] representing the requested table. This method is used - * when we need to create a query plan for a given table. + * Return a [[LogicalPlan]] that represents the given table. * - * This method is different from `getTable`, which only returns the metadata of the table - * in the form of [[CatalogTable]]. The [[LogicalPlan]] returned by this method contains - * the metadata of the table in the form of [[CatalogTable]]. + * If a database is specified in `name`, this will return the table from that database. + * If no database is specified, this will first attempt to return a temporary table with + * the same name, then, if that does not exist, return the table from the current database. */ - def lookupRelation(tableIdent: TableIdentifier, alias: Option[String] = None): LogicalPlan + def lookupRelation( + currentDb: String, + name: TableIdentifier, + alias: Option[String] = None): LogicalPlan - def listTables(specifiedDB: Option[String], currentDB: String): Seq[String] + /** + * List all tables in the current database, including temporary tables. + */ + def listTables(currentDb: String): Seq[TableIdentifier] - def listTables(specifiedDB: Option[String], currentDB: String, pattern: String): Seq[String] + /** + * List all matching tables in the current database, including temporary tables. + */ + def listTables(currentDb: String, pattern: String): Seq[TableIdentifier] // -------------------------------------------------------------------------- // Partitions 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 db34af3d26fc..c9e2f20b2857 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 @@ -211,6 +211,7 @@ case class CatalogTablePartition( * future once we have a better understanding of how we want to handle skewed columns. */ case class CatalogTable( + // TODO: just use TableIdentifier here specifiedDatabase: Option[String], name: String, tableType: CatalogTableType, From 5bf695c686d84df500b36713b2ef86226615f3c6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 14 Mar 2016 17:14:59 -0700 Subject: [PATCH 03/20] Do the same for functions and partitions --- .../spark/sql/catalyst/TableIdentifier.scala | 22 +++- .../sql/catalyst/catalog/SessionCatalog.scala | 111 +++++++++--------- .../sql/catalyst/catalog/interface.scala | 1 + 3 files changed, 76 insertions(+), 58 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala index 4d4e4ded9947..ca959c3ea14b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.catalyst /** - * Identifies a `table` in `database`. If `database` is not defined, the current database is used. + * Identifies a table in a database. + * If `database` is not defined, the current database is used. */ private[sql] case class TableIdentifier(table: String, database: Option[String]) { def this(table: String) = this(table, None) @@ -33,3 +34,22 @@ private[sql] case class TableIdentifier(table: String, database: Option[String]) private[sql] object TableIdentifier { def apply(tableName: String): TableIdentifier = new TableIdentifier(tableName) } + +/** + * Identifies a function in a database. + * If `database` is not defined, the current database is used. + */ +// TODO: reuse some code with TableIdentifier. +private[sql] case class FunctionIdentifier(name: String, database: Option[String]) { + def this(name: String) = this(name, None) + + override def toString: String = quotedString + + def quotedString: String = database.map(db => s"`$db`.`$name`").getOrElse(s"`$name`") + + def unquotedString: String = database.map(db => s"$db.$name").getOrElse(name) +} + +private[sql] object FunctionIdentifier { + def apply(funcName: String): FunctionIdentifier = new FunctionIdentifier(funcName) +} 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 a7cb7ead0c7f..b6210538dd9d 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.catalog import java.util.concurrent.ConcurrentHashMap -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -64,6 +64,8 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { // sessions as their metadata is persisted in the underlying catalog. // ---------------------------------------------------------------------------- + // Methods that interact with metastore tables only. + /** * Create a metastore table in the database specified in `tableDefinition`. * If no such table is specified, create it in the current database. @@ -84,7 +86,9 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { /** * Retrieve the metadata of an existing metastore table. */ - def getTableMetadata(name: TableIdentifier): CatalogTable + def getTable(name: TableIdentifier): CatalogTable + + // Methods that interact with temporary tables and metastore tables. /** * Create a temporary table. @@ -140,24 +144,27 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { */ def listTables(currentDb: String, pattern: String): Seq[TableIdentifier] - // -------------------------------------------------------------------------- + // ---------------------------------------------------------------------------- // Partitions + // ---------------------------------------------------------------------------- // All methods in this category interact directly with the underlying catalog. - // -------------------------------------------------------------------------- - // TODO: We need to figure out how these methods interact with our data source tables. - // For data source tables, we do not store values of partitioning columns in the metastore. - // For now, partition values of a data source table will be automatically discovered - // when we load the table. + // These methods are concerned with only metastore tables. + // ---------------------------------------------------------------------------- + + // TODO: We need to figure out how these methods interact with our data source + // tables. For such tables, we do not store values of partitioning columns in + // the metastore. For now, partition values of a data source table will be + // automatically discovered when we load the table. def createPartitions( - db: String, - table: String, + currentDb: String, + tableName: TableIdentifier, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit def dropPartitions( - db: String, - table: String, + currentDb: String, + tableName: TableIdentifier, parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit @@ -166,8 +173,8 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { * This assumes index i of `specs` corresponds to index i of `newSpecs`. */ def renamePartitions( - db: String, - table: String, + currentDb: String, + tableName: TableIdentifier, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit @@ -179,69 +186,59 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { * this becomes a no-op. */ def alterPartitions( - db: String, - table: String, + currentDb: String, + tableName: TableIdentifier, parts: Seq[CatalogTablePartition]): Unit - def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition + def getPartition( + currentDb: String, + tableName: TableIdentifier, + spec: TablePartitionSpec): CatalogTablePartition - def listPartitions(db: String, table: String): Seq[CatalogTablePartition] + def listPartitions( + currentDb: String, + tableName: TableIdentifier): Seq[CatalogTablePartition] - // -------------------------------------------------------------------------- + // ---------------------------------------------------------------------------- // Functions - // -------------------------------------------------------------------------- + // ---------------------------------------------------------------------------- + // There are two kinds of functions, temporary functions and metastore + // functions (permanent UDFs). Temporary functions are isolated across + // sessions. Metastore functions can be used across multiple sessions as + // their metadata is persisted in the underlying catalog. + // ---------------------------------------------------------------------------- + + // Methods that interact with metastore functions only. - // -------------------------------------------------------------------------- - // Functions: Methods for metastore functions (permanent UDFs). - // -------------------------------------------------------------------------- + def createFunction(currentDb: String, funcDefinition: CatalogFunction): Unit - def createFunction(db: String, funcDefinition: CatalogFunction): Unit + def dropFunction(currentDb: String, funcName: FunctionIdentifier): Unit /** - * Drops a permanent function with the given name from the given database. + * Alter a function whose name that matches the one specified in `funcDefinition`, + * assuming the function exists. + * + * Note: If the underlying implementation does not support altering a certain field, + * this becomes a no-op. */ - def dropFunction(db: String, funcName: String): Unit + def alterFunction(currentDb: String, funcDefinition: CatalogFunction): Unit - // -------------------------------------------------------------------------- - // Functions: Methods for metastore functions (permanent UDFs) or temp functions. - // -------------------------------------------------------------------------- + // Methods that interact with temporary functions and metastore functions. def createTempFunction(funcDefinition: CatalogFunction): Unit - /** - * Drops a temporary function with the given name. - */ // TODO: The reason that we distinguish dropFunction and dropTempFunction is that // Hive has DROP FUNCTION and DROP TEMPORARY FUNCTION. We may want to consolidate // dropFunction and dropTempFunction. def dropTempFunction(funcName: String): Unit def renameFunction( - specifiedDB: Option[String], - currentDB: String, - oldName: String, - newName: String): Unit + currentDb: String, + oldName: FunctionIdentifier, + newName: FunctionIdentifier): Unit - /** - * Alter a function whose name that matches the one specified in `funcDefinition`, - * assuming the function exists. - * - * Note: If the underlying implementation does not support altering a certain field, - * this becomes a no-op. - */ - def alterFunction( - specifiedDB: Option[String], - currentDB: String, - funcDefinition: CatalogFunction): Unit - - def getFunction( - specifiedDB: Option[String], - currentDB: String, - funcName: String): CatalogFunction - - def listFunctions( - specifiedDB: Option[String], - currentDB: String, - pattern: String): Seq[String] + def getFunction(currentDb: String, funcIdentifier: FunctionIdentifier): CatalogFunction + + def listFunctions(currentDb: String, pattern: String): Seq[FunctionIdentifier] } 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 c9e2f20b2857..bf8696f011aa 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 @@ -167,6 +167,7 @@ abstract class ExternalCatalog { * @param name name of the function * @param className fully qualified class name, e.g. "org.apache.spark.util.MyFunc" */ +// TODO: use FunctionIdentifier here. case class CatalogFunction(name: String, className: String) From 39a153c1b5ac495766eed13c5bb5e5f1135a4e4f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 13:53:42 -0700 Subject: [PATCH 04/20] Take into account current database in table methods --- .../sql/catalyst/catalog/SessionCatalog.scala | 26 ++++++++++++++----- .../sql/catalyst/catalog/interface.scala | 1 - 2 files changed, 19 insertions(+), 8 deletions(-) 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 b6210538dd9d..f1cd8e2b5951 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 @@ -64,11 +64,13 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { // sessions as their metadata is persisted in the underlying catalog. // ---------------------------------------------------------------------------- - // Methods that interact with metastore tables only. + // ---------------------------------------------------- + // | Methods that interact with metastore tables only | + // ---------------------------------------------------- /** * Create a metastore table in the database specified in `tableDefinition`. - * If no such table is specified, create it in the current database. + * If no such database is specified, create it in the current database. */ def createTable( currentDb: String, @@ -78,17 +80,23 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { /** * Alter the metadata of an existing metastore table identified by `tableDefinition`. * + * If no database is specified in `tableDefinition`, assume the table is in the + * current database. + * * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterTable(tableDefinition: CatalogTable): Unit + def alterTable(currentDb: String, tableDefinition: CatalogTable): Unit /** * Retrieve the metadata of an existing metastore table. + * If no database is specified, assume the table is in the current database. */ - def getTable(name: TableIdentifier): CatalogTable + def getTable(currentDb: String, name: TableIdentifier): CatalogTable - // Methods that interact with temporary tables and metastore tables. + // ------------------------------------------------------------- + // | Methods that interact with temporary and metastore tables | + // ------------------------------------------------------------- /** * Create a temporary table. @@ -208,7 +216,9 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { // their metadata is persisted in the underlying catalog. // ---------------------------------------------------------------------------- - // Methods that interact with metastore functions only. + // ------------------------------------------------------- + // | Methods that interact with metastore functions only | + // ------------------------------------------------------- def createFunction(currentDb: String, funcDefinition: CatalogFunction): Unit @@ -223,7 +233,9 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { */ def alterFunction(currentDb: String, funcDefinition: CatalogFunction): Unit - // Methods that interact with temporary functions and metastore functions. + // ---------------------------------------------------------------- + // | Methods that interact with temporary and metastore functions | + // ---------------------------------------------------------------- def createTempFunction(funcDefinition: CatalogFunction): Unit 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 bf8696f011aa..a2185e6f35b8 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 @@ -212,7 +212,6 @@ case class CatalogTablePartition( * future once we have a better understanding of how we want to handle skewed columns. */ case class CatalogTable( - // TODO: just use TableIdentifier here specifiedDatabase: Option[String], name: String, tableType: CatalogTableType, From dd1fbaef9f53cb61cf726b95fe2bd1a845afa2c3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 14:22:37 -0700 Subject: [PATCH 05/20] Refactor CatalogTable to use TableIdentifier This is a standalone commit such that in the future we can split it out into a separate patch if preferrable. --- .../catalyst/catalog/InMemoryCatalog.scala | 14 +++++----- .../sql/catalyst/catalog/interface.scala | 8 +++--- .../catalyst/catalog/CatalogTestCases.scala | 6 ++--- .../apache/spark/sql/hive/HiveCatalog.scala | 9 ++++--- .../spark/sql/hive/HiveMetastoreCatalog.scala | 26 +++++++------------ .../org/apache/spark/sql/hive/HiveQl.scala | 14 ++++------ .../spark/sql/hive/client/HiveClient.scala | 2 +- .../sql/hive/client/HiveClientImpl.scala | 6 ++--- .../hive/execution/CreateTableAsSelect.scala | 4 +-- .../hive/execution/CreateViewAsSelect.scala | 4 +-- .../apache/spark/sql/hive/HiveQlSuite.scala | 16 ++++++------ .../sql/hive/MetastoreDataSourcesSuite.scala | 3 +-- .../spark/sql/hive/client/VersionsSuite.scala | 4 +-- 13 files changed, 54 insertions(+), 62 deletions(-) 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 f3fa7958db41..86a5c03ce449 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.catalog import scala.collection.mutable import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier /** @@ -156,12 +157,13 @@ class InMemoryCatalog extends ExternalCatalog { tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = synchronized { requireDbExists(db) - if (existsTable(db, tableDefinition.name)) { + val table = tableDefinition.name.table + if (existsTable(db, table)) { if (!ignoreIfExists) { - throw new AnalysisException(s"Table ${tableDefinition.name} already exists in $db database") + throw new AnalysisException(s"Table $table already exists in $db database") } } else { - catalog(db).tables.put(tableDefinition.name, new TableDesc(tableDefinition)) + catalog(db).tables.put(table, new TableDesc(tableDefinition)) } } @@ -182,14 +184,14 @@ class InMemoryCatalog extends ExternalCatalog { override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { requireTableExists(db, oldName) val oldDesc = catalog(db).tables(oldName) - oldDesc.table = oldDesc.table.copy(name = newName) + oldDesc.table = oldDesc.table.copy(name = TableIdentifier(newName, Some(db))) catalog(db).tables.put(newName, oldDesc) catalog(db).tables.remove(oldName) } override def alterTable(db: String, tableDefinition: CatalogTable): Unit = synchronized { - requireTableExists(db, tableDefinition.name) - catalog(db).tables(tableDefinition.name).table = tableDefinition + requireTableExists(db, tableDefinition.name.table) + catalog(db).tables(tableDefinition.name.table).table = tableDefinition } override def getTable(db: String, table: String): CatalogTable = synchronized { 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 a2185e6f35b8..da6204aa77f4 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.catalog import javax.annotation.Nullable import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} /** @@ -212,8 +213,7 @@ case class CatalogTablePartition( * future once we have a better understanding of how we want to handle skewed columns. */ case class CatalogTable( - specifiedDatabase: Option[String], - name: String, + name: TableIdentifier, tableType: CatalogTableType, storage: CatalogStorageFormat, schema: Seq[CatalogColumn], @@ -227,12 +227,12 @@ case class CatalogTable( viewText: Option[String] = None) { /** Return the database this table was specified to belong to, assuming it exists. */ - def database: String = specifiedDatabase.getOrElse { + def database: String = name.database.getOrElse { throw new AnalysisException(s"table $name did not specify database") } /** Return the fully qualified name of this table, assuming the database was specified. */ - def qualifiedName: String = s"$database.$name" + def qualifiedName: String = name.unquotedString /** Syntactic sugar to update a field in `storage`. */ def withNewStorage( 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 b03ba81b5057..fb6e114329e9 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 @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier /** @@ -89,8 +90,7 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { private def newTable(name: String, db: String): CatalogTable = { CatalogTable( - specifiedDatabase = Some(db), - name = name, + name = TableIdentifier(name, Some(db)), tableType = CatalogTableType.EXTERNAL_TABLE, storage = storageFormat, schema = Seq(CatalogColumn("col1", "int"), CatalogColumn("col2", "string")), @@ -277,7 +277,7 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { } test("get table") { - assert(newBasicCatalog().getTable("db2", "tbl1").name == "tbl1") + assert(newBasicCatalog().getTable("db2", "tbl1").name.table == "tbl1") } test("get table when database/table does not exist") { 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 5185e9aac05f..33b6d29e01a3 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.NoSuchItemException import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.hive.client.HiveClient +import org.apache.spark.sql.catalyst.TableIdentifier /** @@ -73,10 +74,10 @@ private[spark] class HiveCatalog(client: HiveClient) extends ExternalCatalog wit } private def requireDbMatches(db: String, table: CatalogTable): Unit = { - if (table.specifiedDatabase != Some(db)) { + if (table.name.database != Some(db)) { throw new AnalysisException( s"Provided database $db does not much the one specified in the " + - s"table definition (${table.specifiedDatabase.getOrElse("n/a")})") + s"table definition (${table.name.database.getOrElse("n/a")})") } } @@ -160,7 +161,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends ExternalCatalog wit } override def renameTable(db: String, oldName: String, newName: String): Unit = withClient { - val newTable = client.getTable(db, oldName).copy(name = newName) + val newTable = client.getTable(db, oldName).copy(name = TableIdentifier(newName, Some(db))) client.alterTable(oldName, newTable) } @@ -173,7 +174,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends ExternalCatalog wit */ override def alterTable(db: String, tableDefinition: CatalogTable): Unit = withClient { requireDbMatches(db, tableDefinition) - requireTableExists(db, tableDefinition.name) + requireTableExists(db, tableDefinition.name.table) client.alterTable(tableDefinition) } 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 c70510b4834d..b6c78691e482 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 @@ -118,8 +118,8 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte private def getQualifiedTableName(t: CatalogTable): QualifiedTableName = { QualifiedTableName( - t.specifiedDatabase.getOrElse(client.currentDatabase).toLowerCase, - t.name.toLowerCase) + t.name.database.getOrElse(client.currentDatabase).toLowerCase, + t.name.table.toLowerCase) } /** A cache of Spark SQL data source tables that have been accessed. */ @@ -293,8 +293,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte def newSparkSQLSpecificMetastoreTable(): CatalogTable = { CatalogTable( - specifiedDatabase = Option(dbName), - name = tblName, + name = TableIdentifier(tblName, Option(dbName)), tableType = tableType, schema = Nil, storage = CatalogStorageFormat( @@ -314,8 +313,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte assert(relation.partitionSchema.isEmpty) CatalogTable( - specifiedDatabase = Option(dbName), - name = tblName, + name = TableIdentifier(tblName, Option(dbName)), tableType = tableType, storage = CatalogStorageFormat( locationUri = Some(relation.location.paths.map(_.toUri.toString).head), @@ -432,7 +430,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." - case None => SubqueryAlias(table.name, hive.parseSql(viewText)) + case None => SubqueryAlias(table.name.table, hive.parseSql(viewText)) case Some(aliasText) => SubqueryAlias(aliasText, hive.parseSql(viewText)) } } else { @@ -618,9 +616,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) execution.CreateViewAsSelect( - table.copy( - specifiedDatabase = Some(dbName), - name = tblName), + table.copy(name = TableIdentifier(tblName, Some(dbName))), child, allowExisting, replace) @@ -642,7 +638,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte if (hive.convertCTAS && table.storage.serde.isEmpty) { // Do the conversion when spark.sql.hive.convertCTAS is true and the query // does not specify any storage format (file format and storage handler). - if (table.specifiedDatabase.isDefined) { + if (table.name.database.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + "when spark.sql.hive.convertCTAS is set to true.") @@ -650,7 +646,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( - TableIdentifier(desc.name), + TableIdentifier(desc.name.table), conf.defaultDataSourceName, temporary = false, Array.empty[String], @@ -671,9 +667,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) execution.CreateTableAsSelect( - desc.copy( - specifiedDatabase = Some(dbName), - name = tblName), + desc.copy(name = TableIdentifier(tblName, Some(dbName))), child, allowExisting) } @@ -824,7 +818,7 @@ private[hive] case class MetastoreRelation( // We start by constructing an API table as Hive performs several important transformations // internally when converting an API table to a QL table. val tTable = new org.apache.hadoop.hive.metastore.api.Table() - tTable.setTableName(table.name) + tTable.setTableName(table.name.table) tTable.setDbName(table.database) val tableParameters = new java.util.HashMap[String, String]() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 739fbaf4446f..00fc8af5781a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -60,7 +60,7 @@ private[hive] case class CreateTableAsSelect( override def output: Seq[Attribute] = Seq.empty[Attribute] override lazy val resolved: Boolean = - tableDesc.specifiedDatabase.isDefined && + tableDesc.name.database.isDefined && tableDesc.schema.nonEmpty && tableDesc.storage.serde.isDefined && tableDesc.storage.inputFormat.isDefined && @@ -185,13 +185,10 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging properties: Map[String, String], allowExist: Boolean, replace: Boolean): CreateViewAsSelect = { - val TableIdentifier(viewName, dbName) = extractTableIdent(viewNameParts) - + val tableIdentifier = extractTableIdent(viewNameParts) val originalText = query.source - val tableDesc = CatalogTable( - specifiedDatabase = dbName, - name = viewName, + name = tableIdentifier, tableType = CatalogTableType.VIRTUAL_VIEW, schema = schema, storage = CatalogStorageFormat( @@ -356,12 +353,11 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging "TOK_TABLELOCATION", "TOK_TABLEPROPERTIES"), children) - val TableIdentifier(tblName, dbName) = extractTableIdent(tableNameParts) + val tableIdentifier = extractTableIdent(tableNameParts) // TODO add bucket support var tableDesc: CatalogTable = CatalogTable( - specifiedDatabase = dbName, - name = tblName, + name = tableIdentifier, tableType = if (externalTable.isDefined) { CatalogTableType.EXTERNAL_TABLE 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 b32aff25be68..d214e5288eff 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 @@ -91,7 +91,7 @@ private[hive] trait HiveClient { def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean): Unit /** Alter a table whose name matches the one specified in `table`, assuming it exists. */ - final def alterTable(table: CatalogTable): Unit = alterTable(table.name, table) + final def alterTable(table: CatalogTable): Unit = alterTable(table.name.table, table) /** Updates the given table with new metadata, optionally renaming the table. */ def alterTable(tableName: String, table: CatalogTable): 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 c108750c383c..b460220bab26 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 @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression @@ -298,8 +299,7 @@ private[hive] class HiveClientImpl( logDebug(s"Looking up $dbName.$tableName") Option(client.getTable(dbName, tableName, false)).map { h => CatalogTable( - specifiedDatabase = Option(h.getDbName), - name = h.getTableName, + name = TableIdentifier(h.getTableName, Option(h.getDbName)), tableType = h.getTableType match { case HiveTableType.EXTERNAL_TABLE => CatalogTableType.EXTERNAL_TABLE case HiveTableType.MANAGED_TABLE => CatalogTableType.MANAGED_TABLE @@ -639,7 +639,7 @@ private[hive] class HiveClientImpl( } private def toHiveTable(table: CatalogTable): HiveTable = { - val hiveTable = new HiveTable(table.database, table.name) + val hiveTable = new HiveTable(table.database, table.name.table) hiveTable.setTableType(table.tableType match { case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE 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 91425d143554..391e2975d008 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 @@ -38,7 +38,7 @@ case class CreateTableAsSelect( allowExisting: Boolean) extends RunnableCommand { - val tableIdentifier = TableIdentifier(tableDesc.name, Some(tableDesc.database)) + private val tableIdentifier = tableDesc.name override def children: Seq[LogicalPlan] = Seq(query) @@ -93,6 +93,6 @@ case class CreateTableAsSelect( } override def argString: String = { - s"[Database:${tableDesc.database}}, TableName: ${tableDesc.name}, InsertIntoHiveTable]" + s"[Database:${tableDesc.database}}, TableName: ${tableDesc.name.table}, InsertIntoHiveTable]" } } 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 6c2b88eb8c6d..8a1cf2caaaaa 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 @@ -44,7 +44,7 @@ private[hive] case class CreateViewAsSelect( assert(tableDesc.schema == Nil || tableDesc.schema.length == childSchema.length) assert(tableDesc.viewText.isDefined) - val tableIdentifier = TableIdentifier(tableDesc.name, Some(tableDesc.database)) + private val tableIdentifier = tableDesc.name override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] @@ -116,7 +116,7 @@ private[hive] case class CreateViewAsSelect( } val viewText = tableDesc.viewText.get - val viewName = quote(tableDesc.name) + val viewName = quote(tableDesc.name.table) s"SELECT $viewOutput FROM ($viewText) $viewName" } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala index 626550f56c19..1c775db9b616 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala @@ -54,8 +54,8 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { val (desc, exists) = extractTableDesc(s1) assert(exists) - assert(desc.specifiedDatabase == Some("mydb")) - assert(desc.name == "page_view") + assert(desc.name.database == Some("mydb")) + assert(desc.name.table == "page_view") assert(desc.tableType == CatalogTableType.EXTERNAL_TABLE) assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == @@ -100,8 +100,8 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { val (desc, exists) = extractTableDesc(s2) assert(exists) - assert(desc.specifiedDatabase == Some("mydb")) - assert(desc.name == "page_view") + assert(desc.name.database == Some("mydb")) + assert(desc.name.table == "page_view") assert(desc.tableType == CatalogTableType.EXTERNAL_TABLE) assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == @@ -127,8 +127,8 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { val s3 = """CREATE TABLE page_view AS SELECT * FROM src""" val (desc, exists) = extractTableDesc(s3) assert(exists == false) - assert(desc.specifiedDatabase == None) - assert(desc.name == "page_view") + assert(desc.name.database == None) + assert(desc.name.table == "page_view") assert(desc.tableType == CatalogTableType.MANAGED_TABLE) assert(desc.storage.locationUri == None) assert(desc.schema == Seq.empty[CatalogColumn]) @@ -162,8 +162,8 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { | ORDER BY key, value""".stripMargin val (desc, exists) = extractTableDesc(s5) assert(exists == false) - assert(desc.specifiedDatabase == None) - assert(desc.name == "ctas2") + assert(desc.name.database == None) + assert(desc.name.table == "ctas2") assert(desc.tableType == CatalogTableType.MANAGED_TABLE) assert(desc.storage.locationUri == None) assert(desc.schema == Seq.empty[CatalogColumn]) 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 81420fea116b..a80c35cd811d 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 @@ -719,8 +719,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv withTable(tableName) { val schema = StructType(StructField("int", IntegerType, true) :: Nil) val hiveTable = CatalogTable( - specifiedDatabase = Some("default"), - name = tableName, + name = TableIdentifier(tableName, Some("default")), tableType = CatalogTableType.MANAGED_TABLE, schema = Seq.empty, storage = CatalogStorageFormat( 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 6292f6c3af02..3d54da11ade4 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 @@ -22,6 +22,7 @@ import java.io.File import org.apache.hadoop.util.VersionInfo import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal, NamedExpression} import org.apache.spark.sql.catalyst.util.quietly @@ -129,8 +130,7 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: createTable") { val table = CatalogTable( - specifiedDatabase = Option("default"), - name = "src", + name = TableIdentifier("src", Some("default")), tableType = CatalogTableType.MANAGED_TABLE, schema = Seq(CatalogColumn("key", "int")), storage = CatalogStorageFormat( From 2118212a6b5314838d322169c756714d9670d9ac Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 14:33:20 -0700 Subject: [PATCH 06/20] Refactor CatalogFunction to use FunctionIdentifier --- .../apache/spark/sql/catalyst/TableIdentifier.scala | 6 +++--- .../sql/catalyst/catalog/InMemoryCatalog.scala | 12 ++++++------ .../spark/sql/catalyst/catalog/interface.scala | 3 +-- .../sql/catalyst/catalog/CatalogTestCases.scala | 13 ++++++++----- .../spark/sql/hive/client/HiveClientImpl.scala | 11 ++++++----- 5 files changed, 24 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala index ca959c3ea14b..816d85daac6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala @@ -40,14 +40,14 @@ private[sql] object TableIdentifier { * If `database` is not defined, the current database is used. */ // TODO: reuse some code with TableIdentifier. -private[sql] case class FunctionIdentifier(name: String, database: Option[String]) { +private[sql] case class FunctionIdentifier(funcName: String, database: Option[String]) { def this(name: String) = this(name, None) override def toString: String = quotedString - def quotedString: String = database.map(db => s"`$db`.`$name`").getOrElse(s"`$name`") + def quotedString: String = database.map(db => s"`$db`.`$funcName`").getOrElse(s"`$funcName`") - def unquotedString: String = database.map(db => s"$db.$name").getOrElse(name) + def unquotedString: String = database.map(db => s"$db.$funcName").getOrElse(funcName) } private[sql] object FunctionIdentifier { 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 86a5c03ce449..483fbc0e2993 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.catalog import scala.collection.mutable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} /** @@ -294,10 +294,10 @@ class InMemoryCatalog extends ExternalCatalog { override def createFunction(db: String, func: CatalogFunction): Unit = synchronized { requireDbExists(db) - if (existsFunction(db, func.name)) { + if (existsFunction(db, func.name.funcName)) { throw new AnalysisException(s"Function $func already exists in $db database") } else { - catalog(db).functions.put(func.name, func) + catalog(db).functions.put(func.name.funcName, func) } } @@ -308,14 +308,14 @@ class InMemoryCatalog extends ExternalCatalog { override def renameFunction(db: String, oldName: String, newName: String): Unit = synchronized { requireFunctionExists(db, oldName) - val newFunc = getFunction(db, oldName).copy(name = newName) + val newFunc = getFunction(db, oldName).copy(name = FunctionIdentifier(newName, Some(db))) catalog(db).functions.remove(oldName) catalog(db).functions.put(newName, newFunc) } override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = synchronized { - requireFunctionExists(db, funcDefinition.name) - catalog(db).functions.put(funcDefinition.name, funcDefinition) + requireFunctionExists(db, funcDefinition.name.funcName) + catalog(db).functions.put(funcDefinition.name.funcName, funcDefinition) } override def getFunction(db: String, funcName: String): CatalogFunction = synchronized { 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 da6204aa77f4..2d20c5436c35 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 @@ -168,8 +168,7 @@ abstract class ExternalCatalog { * @param name name of the function * @param className fully qualified class name, e.g. "org.apache.spark.util.MyFunc" */ -// TODO: use FunctionIdentifier here. -case class CatalogFunction(name: String, className: String) +case class CatalogFunction(name: FunctionIdentifier, className: String) /** 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 fb6e114329e9..59d531a81088 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 @@ -21,7 +21,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} /** @@ -82,7 +82,7 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { catalog } - private def newFunc(): CatalogFunction = CatalogFunction("funcname", funcClass) + private def newFunc(): CatalogFunction = newFunc("funcName") private def newDb(name: String): CatalogDatabase = { CatalogDatabase(name, name + " description", newUriForDatabase(), Map.empty) @@ -97,7 +97,9 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { partitionColumns = Seq(CatalogColumn("a", "int"), CatalogColumn("b", "string"))) } - private def newFunc(name: String): CatalogFunction = CatalogFunction(name, funcClass) + private def newFunc(name: String): CatalogFunction = { + CatalogFunction(FunctionIdentifier(name, database = None), funcClass) + } /** * Whether the catalog's table partitions equal the ones given. @@ -498,7 +500,8 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { test("get function") { val catalog = newBasicCatalog() - assert(catalog.getFunction("db2", "func1") == newFunc("func1")) + assert(catalog.getFunction("db2", "func1") == + CatalogFunction(FunctionIdentifier("func1", Some("db2")), funcClass)) intercept[AnalysisException] { catalog.getFunction("db2", "does_not_exist") } @@ -517,7 +520,7 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { assert(catalog.getFunction("db2", "func1").className == funcClass) catalog.renameFunction("db2", "func1", newName) intercept[AnalysisException] { catalog.getFunction("db2", "func1") } - assert(catalog.getFunction("db2", newName).name == newName) + assert(catalog.getFunction("db2", newName).name.funcName == newName) assert(catalog.getFunction("db2", newName).className == funcClass) intercept[AnalysisException] { catalog.renameFunction("db2", "does_not_exist", "me") } } 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 b460220bab26..3040ec93f8d8 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 @@ -35,7 +35,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkConf, SparkException} -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression @@ -545,13 +545,13 @@ private[hive] class HiveClientImpl( } override def renameFunction(db: String, oldName: String, newName: String): Unit = withHiveState { - val catalogFunc = getFunction(db, oldName).copy(name = newName) + val catalogFunc = getFunction(db, oldName).copy(name = FunctionIdentifier(newName, Some(db))) val hiveFunc = toHiveFunction(catalogFunc, db) client.alterFunction(db, oldName, hiveFunc) } override def alterFunction(db: String, func: CatalogFunction): Unit = withHiveState { - client.alterFunction(db, func.name, toHiveFunction(func, db)) + client.alterFunction(db, func.name.funcName, toHiveFunction(func, db)) } override def getFunctionOption( @@ -612,7 +612,7 @@ private[hive] class HiveClientImpl( private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { new HiveFunction( - f.name, + f.name.funcName, db, f.className, null, @@ -623,7 +623,8 @@ private[hive] class HiveClientImpl( } private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { - new CatalogFunction(hf.getFunctionName, hf.getClassName) + val name = FunctionIdentifier(hf.getFunctionName, Option(hf.getDbName)) + new CatalogFunction(name, hf.getClassName) } private def toHiveColumn(c: CatalogColumn): FieldSchema = { From 6d530a919c2f61e69d970625f77b99df5c93b019 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 14:38:50 -0700 Subject: [PATCH 07/20] Fix tests --- .../spark/sql/catalyst/catalog/CatalogTestCases.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 59d531a81088..d7df7fb32e0b 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 @@ -78,7 +78,7 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { catalog.createTable("db2", newTable("tbl1", "db2"), ignoreIfExists = false) catalog.createTable("db2", newTable("tbl2", "db2"), ignoreIfExists = false) catalog.createPartitions("db2", "tbl2", Seq(part1, part2), ignoreIfExists = false) - catalog.createFunction("db2", newFunc("func1")) + catalog.createFunction("db2", newFunc("func1", Some("db2"))) catalog } @@ -97,8 +97,8 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { partitionColumns = Seq(CatalogColumn("a", "int"), CatalogColumn("b", "string"))) } - private def newFunc(name: String): CatalogFunction = { - CatalogFunction(FunctionIdentifier(name, database = None), funcClass) + private def newFunc(name: String, database: Option[String] = None): CatalogFunction = { + CatalogFunction(FunctionIdentifier(name, database), funcClass) } /** From 196f7ce1b9cfdcd607e363be10716c2dec409bd2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 14:39:22 -0700 Subject: [PATCH 08/20] Document and clean up function methods --- .../sql/catalyst/catalog/SessionCatalog.scala | 41 +++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) 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 f1cd8e2b5951..1b3bf831ce89 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 @@ -220,13 +220,23 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { // | Methods that interact with metastore functions only | // ------------------------------------------------------- + /** + * Create a metastore function in the database specified in `funcDefinition`. + * If no such database is specified, create it in the current database. + */ def createFunction(currentDb: String, funcDefinition: CatalogFunction): Unit + /** + * Drop a metastore function. + * If no database is specified, assume the function is in the current database. + */ def dropFunction(currentDb: String, funcName: FunctionIdentifier): Unit /** - * Alter a function whose name that matches the one specified in `funcDefinition`, - * assuming the function exists. + * Alter a function whose name that matches the one specified in `funcDefinition`. + * + * If no database is specified in `funcDefinition`, assume the function is in the + * current database. * * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. @@ -237,20 +247,43 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { // | Methods that interact with temporary and metastore functions | // ---------------------------------------------------------------- + /** + * Create a temporary function. + * This assumes no database is specified in `funcDefinition`. + */ def createTempFunction(funcDefinition: CatalogFunction): Unit // TODO: The reason that we distinguish dropFunction and dropTempFunction is that // Hive has DROP FUNCTION and DROP TEMPORARY FUNCTION. We may want to consolidate // dropFunction and dropTempFunction. - def dropTempFunction(funcName: String): Unit + def dropTempFunction(name: String): Unit + /** + * Rename a function. + * + * If a database is specified in `oldName`, this will rename the function in that database. + * If no database is specified, this will first attempt to rename a temporary function with + * the same name, then, if that does not exist, rename the function in the current database. + * + * This assumes the database specified in `oldName` matches the one specified in `newName`. + */ def renameFunction( currentDb: String, oldName: FunctionIdentifier, newName: FunctionIdentifier): Unit - def getFunction(currentDb: String, funcIdentifier: FunctionIdentifier): CatalogFunction + /** + * Retrieve the metadata of an existing function. + * + * If a database is specified in `name`, this will return the function in that database. + * If no database is specified, this will first attempt to return a temporary function with + * the same name, then, if that does not exist, return the function in the current database. + */ + def getFunction(currentDb: String, name: FunctionIdentifier): CatalogFunction + /** + * List all matching functions in the current database, including temporary functions. + */ def listFunctions(currentDb: String, pattern: String): Seq[FunctionIdentifier] } From 5587a4995634af44ceecc9755165eb9a02bc0e5b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 15:32:38 -0700 Subject: [PATCH 09/20] Implement SessionCatalog using ExternalCatalog --- .../sql/catalyst/catalog/SessionCatalog.scala | 236 +++++++++++++++--- .../sql/catalyst/catalog/interface.scala | 19 ++ 2 files changed, 221 insertions(+), 34 deletions(-) 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 1b3bf831ce89..e761b34f8599 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 @@ -19,8 +19,11 @@ package org.apache.spark.sql.catalyst.catalog import java.util.concurrent.ConcurrentHashMap +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{SubqueryAlias, LogicalPlan} /** @@ -28,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan * 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. */ -abstract class SessionCatalog(catalog: ExternalCatalog) { +class SessionCatalog(externalCatalog: ExternalCatalog) { import ExternalCatalog._ private[this] val tempTables = new ConcurrentHashMap[String, LogicalPlan] @@ -41,19 +44,33 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { // All methods in this category interact directly with the underlying catalog. // ---------------------------------------------------------------------------- - def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit + def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = { + externalCatalog.createDatabase(dbDefinition, ignoreIfExists) + } - def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit + def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { + externalCatalog.dropDatabase(db, ignoreIfNotExists, cascade) + } - def alterDatabase(dbDefinition: CatalogDatabase): Unit + def alterDatabase(dbDefinition: CatalogDatabase): Unit = { + externalCatalog.alterDatabase(dbDefinition) + } - def getDatabase(db: String): CatalogDatabase + def getDatabase(db: String): CatalogDatabase = { + externalCatalog.getDatabase(db) + } - def databaseExists(db: String): Boolean + def databaseExists(db: String): Boolean = { + externalCatalog.databaseExists(db) + } - def listDatabases(): Seq[String] + def listDatabases(): Seq[String] = { + externalCatalog.listDatabases() + } - def listDatabases(pattern: String): Seq[String] + def listDatabases(pattern: String): Seq[String] = { + externalCatalog.listDatabases(pattern) + } // ---------------------------------------------------------------------------- // Tables @@ -75,7 +92,12 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { def createTable( currentDb: String, tableDefinition: CatalogTable, - ignoreIfExists: Boolean): Unit + ignoreIfExists: Boolean): Unit = { + val db = tableDefinition.name.database.getOrElse(currentDb) + val newTableDefinition = tableDefinition.copy( + name = TableIdentifier(tableDefinition.name.table, Some(db))) + externalCatalog.createTable(db, newTableDefinition, ignoreIfExists) + } /** * Alter the metadata of an existing metastore table identified by `tableDefinition`. @@ -86,13 +108,21 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterTable(currentDb: String, tableDefinition: CatalogTable): Unit + def alterTable(currentDb: String, tableDefinition: CatalogTable): Unit = { + val db = tableDefinition.name.database.getOrElse(currentDb) + val newTableDefinition = tableDefinition.copy( + name = TableIdentifier(tableDefinition.name.table, Some(db))) + externalCatalog.alterTable(db, newTableDefinition) + } /** * Retrieve the metadata of an existing metastore table. * If no database is specified, assume the table is in the current database. */ - def getTable(currentDb: String, name: TableIdentifier): CatalogTable + def getTable(currentDb: String, name: TableIdentifier): CatalogTable = { + val db = name.database.getOrElse(currentDb) + externalCatalog.getTable(db, name.table) + } // ------------------------------------------------------------- // | Methods that interact with temporary and metastore tables | @@ -100,9 +130,16 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { /** * Create a temporary table. - * If a temporary table with the same name already exists, this throws an exception. */ - def createTempTable(name: String, tableDefinition: LogicalPlan): Unit + def createTempTable( + name: String, + tableDefinition: LogicalPlan, + ignoreIfExists: Boolean): Unit = { + if (tempTables.contains(name) && !ignoreIfExists) { + throw new AnalysisException(s"Temporary table '$name' already exists.") + } + tempTables.put(name, tableDefinition) + } /** * Rename a table. @@ -116,7 +153,15 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { def renameTable( currentDb: String, oldName: TableIdentifier, - newName: TableIdentifier): Unit + newName: TableIdentifier): Unit = { + val db = oldName.database.getOrElse(currentDb) + if (oldName.database.isDefined || !tempTables.contains(oldName.table)) { + externalCatalog.renameTable(db, oldName.table, newName.table) + } else { + val table = tempTables.remove(oldName.table) + tempTables.put(newName.table, table) + } + } /** * Drop a table. @@ -128,7 +173,14 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { def dropTable( currentDb: String, name: TableIdentifier, - ignoreIfNotExists: Boolean): Unit + ignoreIfNotExists: Boolean): Unit = { + val db = name.database.getOrElse(currentDb) + if (name.database.isDefined || !tempTables.contains(name.table)) { + externalCatalog.dropTable(db, name.table, ignoreIfNotExists) + } else { + tempTables.remove(name.table) + } + } /** * Return a [[LogicalPlan]] that represents the given table. @@ -140,17 +192,42 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { def lookupRelation( currentDb: String, name: TableIdentifier, - alias: Option[String] = None): LogicalPlan + alias: Option[String] = None): LogicalPlan = { + val db = name.database.getOrElse(currentDb) + val relation = + if (name.database.isDefined || !tempTables.contains(name.table)) { + val metadata = externalCatalog.getTable(db, name.table) + CatalogRelation(db, metadata, alias) + } else { + tempTables.get(name.table) + } + val tableWithQualifiers = 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, tableWithQualifiers)).getOrElse(tableWithQualifiers) + } /** * List all tables in the current database, including temporary tables. */ - def listTables(currentDb: String): Seq[TableIdentifier] + def listTables(currentDb: String): Seq[TableIdentifier] = { + val tablesInCurrentDb = externalCatalog.listTables(currentDb).map { t => + TableIdentifier(t, Some(currentDb)) + } + val _tempTables = tempTables.keys().asScala.map { t => TableIdentifier(t) } + tablesInCurrentDb ++ _tempTables + } /** * List all matching tables in the current database, including temporary tables. */ - def listTables(currentDb: String, pattern: String): Seq[TableIdentifier] + def listTables(currentDb: String, pattern: String): Seq[TableIdentifier] = { + val tablesInCurrentDb = externalCatalog.listTables(currentDb, pattern).map { t => + TableIdentifier(t, Some(currentDb)) + } + val _tempTables = tempTables.keys().asScala.map { t => TableIdentifier(t) } + tablesInCurrentDb ++ _tempTables + } // ---------------------------------------------------------------------------- // Partitions @@ -164,48 +241,86 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { // the metastore. For now, partition values of a data source table will be // automatically discovered when we load the table. + /** + * Create partitions in an existing table, assuming it exists. + * If no database is specified, assume the table is in the current database. + */ def createPartitions( currentDb: String, tableName: TableIdentifier, parts: Seq[CatalogTablePartition], - ignoreIfExists: Boolean): Unit + ignoreIfExists: Boolean): Unit = { + val db = tableName.database.getOrElse(currentDb) + externalCatalog.createPartitions(db, tableName.table, parts, ignoreIfExists) + } + /** + * Drop partitions from a table, assuming they exist. + * If no database is specified, assume the table is in the current database. + */ def dropPartitions( currentDb: String, tableName: TableIdentifier, parts: Seq[TablePartitionSpec], - ignoreIfNotExists: Boolean): Unit + ignoreIfNotExists: Boolean): Unit = { + val db = tableName.database.getOrElse(currentDb) + externalCatalog.dropPartitions(db, tableName.table, parts, ignoreIfNotExists) + } /** * Override the specs of one or many existing table partitions, assuming they exist. + * * This assumes index i of `specs` corresponds to index i of `newSpecs`. + * If no database is specified, assume the table is in the current database. */ def renamePartitions( currentDb: String, tableName: TableIdentifier, specs: Seq[TablePartitionSpec], - newSpecs: Seq[TablePartitionSpec]): Unit + newSpecs: Seq[TablePartitionSpec]): Unit = { + val db = tableName.database.getOrElse(currentDb) + externalCatalog.renamePartitions(db, tableName.table, specs, newSpecs) + } /** * Alter one or many table partitions whose specs that match those specified in `parts`, * assuming the partitions exist. * + * If no database is specified, assume the table is in the current database. + * * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ def alterPartitions( currentDb: String, tableName: TableIdentifier, - parts: Seq[CatalogTablePartition]): Unit + parts: Seq[CatalogTablePartition]): Unit = { + val db = tableName.database.getOrElse(currentDb) + externalCatalog.alterPartitions(db, tableName.table, parts) + } + /** + * Retrieve the metadata of a table partition, assuming it exists. + * If no database is specified, assume the table is in the current database. + */ def getPartition( currentDb: String, tableName: TableIdentifier, - spec: TablePartitionSpec): CatalogTablePartition + spec: TablePartitionSpec): CatalogTablePartition = { + val db = tableName.database.getOrElse(currentDb) + externalCatalog.getPartition(db, tableName.table, spec) + } + /** + * List all partitions in a table, assuming it exists. + * If no database is specified, assume the table is in the current database. + */ def listPartitions( currentDb: String, - tableName: TableIdentifier): Seq[CatalogTablePartition] + tableName: TableIdentifier): Seq[CatalogTablePartition] = { + val db = tableName.database.getOrElse(currentDb) + externalCatalog.listPartitions(db, tableName.table) + } // ---------------------------------------------------------------------------- // Functions @@ -224,16 +339,24 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { * Create a metastore function in the database specified in `funcDefinition`. * If no such database is specified, create it in the current database. */ - def createFunction(currentDb: String, funcDefinition: CatalogFunction): Unit + def createFunction(currentDb: String, funcDefinition: CatalogFunction): Unit = { + val db = funcDefinition.name.database.getOrElse(currentDb) + val newFuncDefinition = funcDefinition.copy( + name = FunctionIdentifier(funcDefinition.name.funcName, Some(db))) + externalCatalog.createFunction(db, newFuncDefinition) + } /** * Drop a metastore function. * If no database is specified, assume the function is in the current database. */ - def dropFunction(currentDb: String, funcName: FunctionIdentifier): Unit + def dropFunction(currentDb: String, name: FunctionIdentifier): Unit = { + val db = name.database.getOrElse(currentDb) + externalCatalog.dropFunction(db, name.funcName) + } /** - * Alter a function whose name that matches the one specified in `funcDefinition`. + * Alter a metastore function whose name that matches the one specified in `funcDefinition`. * * If no database is specified in `funcDefinition`, assume the function is in the * current database. @@ -241,7 +364,12 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterFunction(currentDb: String, funcDefinition: CatalogFunction): Unit + def alterFunction(currentDb: String, funcDefinition: CatalogFunction): Unit = { + val db = funcDefinition.name.database.getOrElse(currentDb) + val newFuncDefinition = funcDefinition.copy( + name = FunctionIdentifier(funcDefinition.name.funcName, Some(db))) + externalCatalog.alterFunction(db, newFuncDefinition) + } // ---------------------------------------------------------------- // | Methods that interact with temporary and metastore functions | @@ -251,12 +379,31 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { * Create a temporary function. * This assumes no database is specified in `funcDefinition`. */ - def createTempFunction(funcDefinition: CatalogFunction): Unit + def createTempFunction( + funcDefinition: CatalogFunction, + ignoreIfExists: Boolean): Unit = { + require(funcDefinition.name.database.isEmpty, + "attempted to create a temporary function while specifying a database") + val name = funcDefinition.name.funcName + if (tempFunctions.contains(name) && !ignoreIfExists) { + throw new AnalysisException(s"Temporary function '$name' already exists.") + } + tempFunctions.put(name, funcDefinition) + } + /** + * Drop a temporary function. + */ // TODO: The reason that we distinguish dropFunction and dropTempFunction is that // Hive has DROP FUNCTION and DROP TEMPORARY FUNCTION. We may want to consolidate // dropFunction and dropTempFunction. - def dropTempFunction(name: String): Unit + def dropTempFunction(name: String, ignoreIfNotExists: Boolean): Unit = { + if (!tempFunctions.contains(name) && !ignoreIfNotExists) { + throw new AnalysisException( + s"Temporary function '$name' cannot be dropped because it does not exist!") + } + tempFunctions.remove(name) + } /** * Rename a function. @@ -270,7 +417,16 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { def renameFunction( currentDb: String, oldName: FunctionIdentifier, - newName: FunctionIdentifier): Unit + newName: FunctionIdentifier): Unit = { + val db = oldName.database.getOrElse(currentDb) + if (oldName.database.isDefined || !tempFunctions.contains(oldName.funcName)) { + externalCatalog.renameFunction(db, oldName.funcName, newName.funcName) + } else { + val func = tempFunctions.remove(oldName.funcName) + val newFunc = func.copy(name = func.name.copy(funcName = newName.funcName)) + tempFunctions.put(newName.funcName, newFunc) + } + } /** * Retrieve the metadata of an existing function. @@ -279,11 +435,23 @@ abstract class SessionCatalog(catalog: ExternalCatalog) { * If no database is specified, this will first attempt to return a temporary function with * the same name, then, if that does not exist, return the function in the current database. */ - def getFunction(currentDb: String, name: FunctionIdentifier): CatalogFunction + def getFunction(currentDb: String, name: FunctionIdentifier): CatalogFunction = { + val db = name.database.getOrElse(currentDb) + if (name.database.isDefined || !tempFunctions.contains(name.funcName)) { + externalCatalog.getFunction(db, name.funcName) + } else { + tempFunctions.get(name.funcName) + } + } /** * List all matching functions in the current database, including temporary functions. */ - def listFunctions(currentDb: String, pattern: String): Seq[FunctionIdentifier] + def listFunctions(currentDb: String, pattern: String): Seq[FunctionIdentifier] = { + val functionsInCurrentDb = externalCatalog.listFunctions(currentDb, pattern).map { f => + FunctionIdentifier(f, Some(currentDb)) + } + functionsInCurrentDb ++ tempFunctions.keys.asScala.map { f => FunctionIdentifier(f) } + } } 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 2d20c5436c35..5e4460e9a744 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 @@ -21,6 +21,8 @@ import javax.annotation.Nullable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} /** @@ -272,3 +274,20 @@ object ExternalCatalog { */ type TablePartitionSpec = Map[String, String] } + + +/** + * A [[LogicalPlan]] that wraps [[CatalogTable]]. + */ +case class CatalogRelation( + db: String, + metadata: CatalogTable, + alias: Option[String]) + extends LeafNode { + + // TODO: implement this + override def output: Seq[Attribute] = Seq.empty + + require(metadata.name.database == Some(db), + "provided database does not much the one specified in the table definition") +} From 90ccdbb22bd8baf8caf839047148ebfd326b3593 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 15:33:30 -0700 Subject: [PATCH 10/20] Fix style --- .../org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 e761b34f8599..147ce02f7000 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 @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.plans.logical.{SubqueryAlias, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} /** 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 33b6d29e01a3..439501fe1986 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 @@ -24,10 +24,10 @@ import org.apache.thrift.TException import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchItemException import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.hive.client.HiveClient -import org.apache.spark.sql.catalyst.TableIdentifier /** From caa4013e457a46ef0b8c3a2291cb375eb9064972 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 15:44:23 -0700 Subject: [PATCH 11/20] Clean up duplicate code in Table/FunctionIdentifier --- ...ableIdentifier.scala => identifiers.scala} | 43 ++++++++++++------- 1 file changed, 27 insertions(+), 16 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/{TableIdentifier.scala => identifiers.scala} (62%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala similarity index 62% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index 816d85daac6b..23090aa6f676 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -17,37 +17,48 @@ package org.apache.spark.sql.catalyst + /** - * Identifies a table in a database. - * If `database` is not defined, the current database is used. + * An identifier that optionally specifies a database. + * + * Format (unquoted): "name" or "db.name" + * Format (quoted): "`name`" or "`db`.`name`" */ -private[sql] case class TableIdentifier(table: String, database: Option[String]) { - def this(table: String) = this(table, None) - +private[sql] abstract class IdentifierWithDatabase(name: String) { + def database: Option[String] + def quotedString: String = database.map(db => s"`$db`.`$name`").getOrElse(s"`$name`") + def unquotedString: String = database.map(db => s"$db.$name").getOrElse(name) override def toString: String = quotedString +} + - def quotedString: String = database.map(db => s"`$db`.`$table`").getOrElse(s"`$table`") +/** + * Identifies a table in a database. + * If `database` is not defined, the current database is used. + */ +private[sql] case class TableIdentifier( + table: String, + database: Option[String]) + extends IdentifierWithDatabase(table) { - def unquotedString: String = database.map(db => s"$db.$table").getOrElse(table) + def this(name: String) = this(name, None) } private[sql] object TableIdentifier { - def apply(tableName: String): TableIdentifier = new TableIdentifier(tableName) + def apply(tableName: String): TableIdentifier = TableIdentifier(tableName) } + /** * Identifies a function in a database. * If `database` is not defined, the current database is used. */ -// TODO: reuse some code with TableIdentifier. -private[sql] case class FunctionIdentifier(funcName: String, database: Option[String]) { - def this(name: String) = this(name, None) +private[sql] case class FunctionIdentifier( + funcName: String, + database: Option[String]) + extends IdentifierWithDatabase(funcName) { - override def toString: String = quotedString - - def quotedString: String = database.map(db => s"`$db`.`$funcName`").getOrElse(s"`$funcName`") - - def unquotedString: String = database.map(db => s"$db.$funcName").getOrElse(funcName) + def this(name: String) = this(name, None) } private[sql] object FunctionIdentifier { From d3f252d4d21b91a22dd7277f983f84daa56d65b5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 16:12:55 -0700 Subject: [PATCH 12/20] Refactor CatalogTestCases to make methods accessible --- .../catalyst/catalog/CatalogTestCases.scala | 160 ++++++++++-------- .../catalog/InMemoryCatalogSuite.scala | 9 +- .../catalog/SessionCatalogSuite.scala | 29 ++++ .../spark/sql/hive/HiveCatalogSuite.scala | 13 +- 4 files changed, 129 insertions(+), 82 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala 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 d7df7fb32e0b..2b12504bab86 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 @@ -22,6 +22,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.util.Utils /** @@ -30,23 +31,10 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} * Implementations of the [[ExternalCatalog]] interface can create test suites by extending this. */ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { - private lazy val storageFormat = CatalogStorageFormat( - locationUri = None, - inputFormat = Some(tableInputFormat), - outputFormat = Some(tableOutputFormat), - serde = None, - serdeProperties = Map.empty) - private lazy val part1 = CatalogTablePartition(Map("a" -> "1", "b" -> "2"), storageFormat) - private lazy val part2 = CatalogTablePartition(Map("a" -> "3", "b" -> "4"), storageFormat) - private lazy val part3 = CatalogTablePartition(Map("a" -> "5", "b" -> "6"), storageFormat) - private val funcClass = "org.apache.spark.myFunc" - - // Things subclasses should override - protected val tableInputFormat: String = "org.apache.park.serde.MyInputFormat" - protected val tableOutputFormat: String = "org.apache.park.serde.MyOutputFormat" - protected def newUriForDatabase(): String = "uri" + protected val utils: CatalogTestUtils + import utils._ + protected def resetState(): Unit = { } - protected def newEmptyCatalog(): ExternalCatalog // Clear all state after each test override def afterEach(): Unit = { @@ -57,63 +45,6 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { } } - /** - * Creates a basic catalog, with the following structure: - * - * default - * db1 - * db2 - * - tbl1 - * - tbl2 - * - part1 - * - part2 - * - func1 - */ - private def newBasicCatalog(): ExternalCatalog = { - val catalog = newEmptyCatalog() - // When testing against a real catalog, the default database may already exist - catalog.createDatabase(newDb("default"), ignoreIfExists = true) - catalog.createDatabase(newDb("db1"), ignoreIfExists = false) - catalog.createDatabase(newDb("db2"), ignoreIfExists = false) - catalog.createTable("db2", newTable("tbl1", "db2"), ignoreIfExists = false) - catalog.createTable("db2", newTable("tbl2", "db2"), ignoreIfExists = false) - catalog.createPartitions("db2", "tbl2", Seq(part1, part2), ignoreIfExists = false) - catalog.createFunction("db2", newFunc("func1", Some("db2"))) - catalog - } - - private def newFunc(): CatalogFunction = newFunc("funcName") - - private def newDb(name: String): CatalogDatabase = { - CatalogDatabase(name, name + " description", newUriForDatabase(), Map.empty) - } - - private def newTable(name: String, db: String): CatalogTable = { - CatalogTable( - name = TableIdentifier(name, Some(db)), - tableType = CatalogTableType.EXTERNAL_TABLE, - storage = storageFormat, - schema = Seq(CatalogColumn("col1", "int"), CatalogColumn("col2", "string")), - partitionColumns = Seq(CatalogColumn("a", "int"), CatalogColumn("b", "string"))) - } - - private def newFunc(name: String, database: Option[String] = None): CatalogFunction = { - CatalogFunction(FunctionIdentifier(name, database), funcClass) - } - - /** - * Whether the catalog's table partitions equal the ones given. - * Note: Hive sets some random serde things, so we just compare the specs here. - */ - private def catalogPartitionsEqual( - catalog: ExternalCatalog, - db: String, - table: String, - parts: Seq[CatalogTablePartition]): Boolean = { - catalog.listPartitions(db, table).map(_.spec).toSet == parts.map(_.spec).toSet - } - - // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -556,3 +487,86 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { } } + + +/** + * A collection of utility fields and methods for tests related to the [[ExternalCatalog]]. + */ +abstract class CatalogTestUtils { + + // Unimplemented methods + val tableInputFormat: String + val tableOutputFormat: String + def newEmptyCatalog(): ExternalCatalog + + // These fields must be lazy because they rely on fields that are not implemented yet + lazy val storageFormat = CatalogStorageFormat( + locationUri = None, + inputFormat = Some(tableInputFormat), + outputFormat = Some(tableOutputFormat), + serde = None, + serdeProperties = Map.empty) + lazy val part1 = CatalogTablePartition(Map("a" -> "1", "b" -> "2"), storageFormat) + lazy val part2 = CatalogTablePartition(Map("a" -> "3", "b" -> "4"), storageFormat) + lazy val part3 = CatalogTablePartition(Map("a" -> "5", "b" -> "6"), storageFormat) + lazy val funcClass = "org.apache.spark.myFunc" + + /** + * Creates a basic catalog, with the following structure: + * + * default + * db1 + * db2 + * - tbl1 + * - tbl2 + * - part1 + * - part2 + * - func1 + */ + def newBasicCatalog(): ExternalCatalog = { + val catalog = newEmptyCatalog() + // When testing against a real catalog, the default database may already exist + catalog.createDatabase(newDb("default"), ignoreIfExists = true) + catalog.createDatabase(newDb("db1"), ignoreIfExists = false) + catalog.createDatabase(newDb("db2"), ignoreIfExists = false) + catalog.createTable("db2", newTable("tbl1", "db2"), ignoreIfExists = false) + catalog.createTable("db2", newTable("tbl2", "db2"), ignoreIfExists = false) + catalog.createPartitions("db2", "tbl2", Seq(part1, part2), ignoreIfExists = false) + catalog.createFunction("db2", newFunc("func1", Some("db2"))) + catalog + } + + def newFunc(): CatalogFunction = newFunc("funcName") + + def newUriForDatabase(): String = Utils.createTempDir().getAbsolutePath + + def newDb(name: String): CatalogDatabase = { + CatalogDatabase(name, name + " description", newUriForDatabase(), Map.empty) + } + + def newTable(name: String, db: String): CatalogTable = { + CatalogTable( + name = TableIdentifier(name, Some(db)), + tableType = CatalogTableType.EXTERNAL_TABLE, + storage = storageFormat, + schema = Seq(CatalogColumn("col1", "int"), CatalogColumn("col2", "string")), + partitionColumns = Seq(CatalogColumn("a", "int"), CatalogColumn("b", "string"))) + } + + def newFunc(name: String, database: Option[String] = None): CatalogFunction = { + CatalogFunction(FunctionIdentifier(name, database), funcClass) + } + + /** + * Whether the catalog's table partitions equal the ones given. + * Note: Hive sets some random serde things, so we just compare the specs here. + */ + def catalogPartitionsEqual( + catalog: ExternalCatalog, + db: String, + table: String, + parts: Seq[CatalogTablePartition]): Boolean = { + catalog.listPartitions(db, table).map(_.spec).toSet == parts.map(_.spec).toSet + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala index 9531758ffd59..63a7b2c661ec 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala @@ -17,7 +17,14 @@ package org.apache.spark.sql.catalyst.catalog + /** Test suite for the [[InMemoryCatalog]]. */ class InMemoryCatalogSuite extends CatalogTestCases { - override protected def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog + + protected override val utils: CatalogTestUtils = new CatalogTestUtils { + override val tableInputFormat: String = "org.apache.park.SequenceFileInputFormat" + override val tableOutputFormat: String = "org.apache.park.SequenceFileOutputFormat" + override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog + } + } 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 new file mode 100644 index 000000000000..11c8e9a52fb6 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -0,0 +1,29 @@ +/* + * 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.catalog + +import org.apache.spark.SparkFunSuite + + +class SessionCatalogSuite extends SparkFunSuite { + + test("dummy") { + + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala index 2809f9439b82..0dc4fea22db7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala @@ -36,15 +36,12 @@ class HiveCatalogSuite extends CatalogTestCases { sparkConf = new SparkConf()).createClient() } - protected override val tableInputFormat: String = - "org.apache.hadoop.mapred.SequenceFileInputFormat" - protected override val tableOutputFormat: String = - "org.apache.hadoop.mapred.SequenceFileOutputFormat" - - protected override def newUriForDatabase(): String = Utils.createTempDir().getAbsolutePath + protected override val utils: CatalogTestUtils = new CatalogTestUtils { + override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" + override val tableOutputFormat: String = "org.apache.hadoop.mapred.SequenceFileOutputFormat" + override def newEmptyCatalog(): ExternalCatalog = new HiveCatalog(client) + } protected override def resetState(): Unit = client.reset() - protected override def newEmptyCatalog(): ExternalCatalog = new HiveCatalog(client) - } From 2f5121b43c938b2b585de0c3d80680c0ad5a8a7d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 16:59:38 -0700 Subject: [PATCH 13/20] Fix infinite loop (woops) --- .../main/scala/org/apache/spark/sql/catalyst/identifiers.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index 23090aa6f676..54599028888b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -45,7 +45,7 @@ private[sql] case class TableIdentifier( } private[sql] object TableIdentifier { - def apply(tableName: String): TableIdentifier = TableIdentifier(tableName) + def apply(tableName: String): TableIdentifier = new TableIdentifier(tableName) } From 794744565269bb9ffb00f8d7a81d7b703251f956 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 18:52:30 -0700 Subject: [PATCH 14/20] Implement tests for databases and tables --- .../catalyst/catalog/InMemoryCatalog.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 32 +- .../sql/catalyst/catalog/interface.scala | 2 +- .../catalog/SessionCatalogSuite.scala | 346 +++++++++++++++++- 4 files changed, 370 insertions(+), 12 deletions(-) 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 483fbc0e2993..e476b319da12 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 @@ -176,7 +176,7 @@ class InMemoryCatalog extends ExternalCatalog { catalog(db).tables.remove(table) } else { if (!ignoreIfNotExists) { - throw new AnalysisException(s"Table $table does not exist in $db database") + throw new AnalysisException(s"Table '$table' does not exist in database '$db'") } } } 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 147ce02f7000..2a8f2cd21e1f 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 @@ -135,7 +135,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { name: String, tableDefinition: LogicalPlan, ignoreIfExists: Boolean): Unit = { - if (tempTables.contains(name) && !ignoreIfExists) { + if (tempTables.containsKey(name) && !ignoreIfExists) { throw new AnalysisException(s"Temporary table '$name' already exists.") } tempTables.put(name, tableDefinition) @@ -154,8 +154,11 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { currentDb: String, oldName: TableIdentifier, newName: TableIdentifier): Unit = { + if (oldName.database != newName.database) { + throw new AnalysisException("rename does not support moving tables across databases") + } val db = oldName.database.getOrElse(currentDb) - if (oldName.database.isDefined || !tempTables.contains(oldName.table)) { + if (oldName.database.isDefined || !tempTables.containsKey(oldName.table)) { externalCatalog.renameTable(db, oldName.table, newName.table) } else { val table = tempTables.remove(oldName.table) @@ -175,7 +178,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { name: TableIdentifier, ignoreIfNotExists: Boolean): Unit = { val db = name.database.getOrElse(currentDb) - if (name.database.isDefined || !tempTables.contains(name.table)) { + if (name.database.isDefined || !tempTables.containsKey(name.table)) { externalCatalog.dropTable(db, name.table, ignoreIfNotExists) } else { tempTables.remove(name.table) @@ -195,7 +198,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { alias: Option[String] = None): LogicalPlan = { val db = name.database.getOrElse(currentDb) val relation = - if (name.database.isDefined || !tempTables.contains(name.table)) { + if (name.database.isDefined || !tempTables.containsKey(name.table)) { val metadata = externalCatalog.getTable(db, name.table) CatalogRelation(db, metadata, alias) } else { @@ -225,10 +228,21 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { val tablesInCurrentDb = externalCatalog.listTables(currentDb, pattern).map { t => TableIdentifier(t, Some(currentDb)) } - val _tempTables = tempTables.keys().asScala.map { t => TableIdentifier(t) } + val regex = pattern.replaceAll("\\*", ".*").r + val _tempTables = tempTables.keys().asScala + .filter { t => regex.pattern.matcher(t).matches() } + .map { t => TableIdentifier(t) } tablesInCurrentDb ++ _tempTables } + /** + * Return a temporary table exactly as it was stored. + * For testing only. + */ + private[catalog] def getTempTable(name: String): Option[LogicalPlan] = { + Option(tempTables.get(name)) + } + // ---------------------------------------------------------------------------- // Partitions // ---------------------------------------------------------------------------- @@ -385,7 +399,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { require(funcDefinition.name.database.isEmpty, "attempted to create a temporary function while specifying a database") val name = funcDefinition.name.funcName - if (tempFunctions.contains(name) && !ignoreIfExists) { + if (tempFunctions.containsKey(name) && !ignoreIfExists) { throw new AnalysisException(s"Temporary function '$name' already exists.") } tempFunctions.put(name, funcDefinition) @@ -398,7 +412,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { // Hive has DROP FUNCTION and DROP TEMPORARY FUNCTION. We may want to consolidate // dropFunction and dropTempFunction. def dropTempFunction(name: String, ignoreIfNotExists: Boolean): Unit = { - if (!tempFunctions.contains(name) && !ignoreIfNotExists) { + if (!tempFunctions.containsKey(name) && !ignoreIfNotExists) { throw new AnalysisException( s"Temporary function '$name' cannot be dropped because it does not exist!") } @@ -419,7 +433,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { oldName: FunctionIdentifier, newName: FunctionIdentifier): Unit = { val db = oldName.database.getOrElse(currentDb) - if (oldName.database.isDefined || !tempFunctions.contains(oldName.funcName)) { + if (oldName.database.isDefined || !tempFunctions.containsKey(oldName.funcName)) { externalCatalog.renameFunction(db, oldName.funcName, newName.funcName) } else { val func = tempFunctions.remove(oldName.funcName) @@ -437,7 +451,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { */ def getFunction(currentDb: String, name: FunctionIdentifier): CatalogFunction = { val db = name.database.getOrElse(currentDb) - if (name.database.isDefined || !tempFunctions.contains(name.funcName)) { + if (name.database.isDefined || !tempFunctions.containsKey(name.funcName)) { externalCatalog.getFunction(db, name.funcName) } else { tempFunctions.get(name.funcName) 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 5e4460e9a744..c4e49614c5c3 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 @@ -282,7 +282,7 @@ object ExternalCatalog { case class CatalogRelation( db: String, metadata: CatalogTable, - alias: Option[String]) + alias: Option[String] = None) extends LeafNode { // TODO: implement this 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 11c8e9a52fb6..51b35b2a272c 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 @@ -18,12 +18,356 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias} +/** + * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented. + * + * Note: many of the methods here are very similar to the ones in [[CatalogTestCases]]. + * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method + * signatures but do not extend a common parent. This is largely by design but + * unfortunately leads to very similar test code in two places. + */ class SessionCatalogSuite extends SparkFunSuite { + private val utils = new CatalogTestUtils { + override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat" + override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat" + override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog + } + + import utils._ + + // -------------------------------------------------------------------------- + // Databases + // -------------------------------------------------------------------------- + + test("basic create and list databases") { + val catalog = new SessionCatalog(newEmptyCatalog()) + catalog.createDatabase(newDb("default"), ignoreIfExists = true) + assert(catalog.databaseExists("default")) + assert(!catalog.databaseExists("testing")) + assert(!catalog.databaseExists("testing2")) + catalog.createDatabase(newDb("testing"), ignoreIfExists = false) + assert(catalog.databaseExists("testing")) + assert(catalog.listDatabases().toSet == Set("default", "testing")) + catalog.createDatabase(newDb("testing2"), ignoreIfExists = false) + assert(catalog.listDatabases().toSet == Set("default", "testing", "testing2")) + assert(catalog.databaseExists("testing2")) + assert(!catalog.databaseExists("does_not_exist")) + } + + test("get database when a database exists") { + val catalog = new SessionCatalog(newBasicCatalog()) + val db1 = catalog.getDatabase("db1") + assert(db1.name == "db1") + assert(db1.description.contains("db1")) + } + + test("get database should throw exception when the database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { catalog.getDatabase("db_that_does_not_exist") } + } + + test("list databases without pattern") { + val catalog = new SessionCatalog(newBasicCatalog()) + assert(catalog.listDatabases().toSet == Set("default", "db1", "db2")) + } + + test("list databases with pattern") { + val catalog = new SessionCatalog(newBasicCatalog()) + assert(catalog.listDatabases("db").toSet == Set.empty) + assert(catalog.listDatabases("db*").toSet == Set("db1", "db2")) + assert(catalog.listDatabases("*1").toSet == Set("db1")) + assert(catalog.listDatabases("db2").toSet == Set("db2")) + } + + test("drop database") { + val catalog = new SessionCatalog(newBasicCatalog()) + catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false) + assert(catalog.listDatabases().toSet == Set("default", "db2")) + } + + test("drop database when the database is not empty") { + // Throw exception if there are functions left + val externalCatalog1 = newBasicCatalog() + val sessionCatalog1 = new SessionCatalog(externalCatalog1) + externalCatalog1.dropTable("db2", "tbl1", ignoreIfNotExists = false) + externalCatalog1.dropTable("db2", "tbl2", ignoreIfNotExists = false) + intercept[AnalysisException] { + sessionCatalog1.dropDatabase("db2", ignoreIfNotExists = false, cascade = false) + } + + // Throw exception if there are tables left + val externalCatalog2 = newBasicCatalog() + val sessionCatalog2 = new SessionCatalog(externalCatalog2) + externalCatalog2.dropFunction("db2", "func1") + intercept[AnalysisException] { + sessionCatalog2.dropDatabase("db2", ignoreIfNotExists = false, cascade = false) + } + + // When cascade is true, it should drop them + val externalCatalog3 = newBasicCatalog() + val sessionCatalog3 = new SessionCatalog(externalCatalog3) + externalCatalog3.dropDatabase("db2", ignoreIfNotExists = false, cascade = true) + assert(sessionCatalog3.listDatabases().toSet == Set("default", "db1")) + } + + test("drop database when the database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = false, cascade = false) + } + catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = true, cascade = false) + } + + test("alter database") { + val catalog = new SessionCatalog(newBasicCatalog()) + val db1 = catalog.getDatabase("db1") + // Note: alter properties here because Hive does not support altering other fields + catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true"))) + val newDb1 = catalog.getDatabase("db1") + assert(db1.properties.isEmpty) + assert(newDb1.properties.size == 2) + assert(newDb1.properties.get("k") == Some("v3")) + assert(newDb1.properties.get("good") == Some("true")) + } + + test("alter database should throw exception when the database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.alterDatabase(newDb("does_not_exist")) + } + } + + // -------------------------------------------------------------------------- + // Tables + // -------------------------------------------------------------------------- - test("dummy") { + test("create temporary table") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempTable1 = Range(1, 10, 1, 10, Seq()) + val tempTable2 = Range(1, 20, 2, 10, Seq()) + catalog.createTempTable("tbl1", tempTable1, ignoreIfExists = false) + catalog.createTempTable("tbl2", tempTable2, ignoreIfExists = false) + assert(catalog.getTempTable("tbl1") == Some(tempTable1)) + assert(catalog.getTempTable("tbl2") == Some(tempTable2)) + assert(catalog.getTempTable("tbl3") == None) + // Temporary table already exists + intercept[AnalysisException] { + catalog.createTempTable("tbl1", tempTable1, ignoreIfExists = false) + } + // Temporary table already exists but we override it + catalog.createTempTable("tbl1", tempTable2, ignoreIfExists = true) + assert(catalog.getTempTable("tbl1") == Some(tempTable2)) + } + + test("drop table") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + sessionCatalog.dropTable("db2", TableIdentifier("tbl1"), ignoreIfNotExists = false) + assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) + sessionCatalog.dropTable( + "db_not_read", TableIdentifier("tbl2", Some("db2")), ignoreIfNotExists = false) + assert(externalCatalog.listTables("db2").isEmpty) + } + + test("drop table when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + // Should always throw exception when the database does not exist + intercept[AnalysisException] { + catalog.dropTable("unknown_db", TableIdentifier("unknown_table"), ignoreIfNotExists = false) + } + intercept[AnalysisException] { + catalog.dropTable("unknown_db", TableIdentifier("unknown_table"), ignoreIfNotExists = true) + } + + // Should throw exception when the table does not exist, if ignoreIfNotExists is false + intercept[AnalysisException] { + catalog.dropTable("db2", TableIdentifier("unknown_table"), ignoreIfNotExists = false) + } + catalog.dropTable("db2", TableIdentifier("unknown_table"), ignoreIfNotExists = true) + } + + test("drop temp table") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val tempTable = Range(1, 10, 2, 10, Seq()) + sessionCatalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) + assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + // If database is not specified, temp table should be dropped first + sessionCatalog.dropTable("db_not_read", TableIdentifier("tbl1"), ignoreIfNotExists = false) + assert(sessionCatalog.getTempTable("tbl1") == None) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + // If database is specified, temp tables are never dropped + sessionCatalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) + sessionCatalog.dropTable( + "db_not_read", TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false) + assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) + assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) + } + + test("rename table") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + sessionCatalog.renameTable("db2", TableIdentifier("tbl1"), TableIdentifier("tblone")) + assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbl2")) + sessionCatalog.renameTable( + "db_not_read", TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo", Some("db2"))) + assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbltwo")) + } + + test("rename table when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.renameTable( + "unknown_db", TableIdentifier("unknown_table"), TableIdentifier("unknown_table")) + } + intercept[AnalysisException] { + catalog.renameTable( + "db2", TableIdentifier("unknown_table"), TableIdentifier("unknown_table")) + } + // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match + intercept[AnalysisException] { + catalog.renameTable( + "db_not_read", TableIdentifier("tblone", Some("db2")), TableIdentifier("x", Some("db1"))) + } + } + + test("rename temp table") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val tempTable = Range(1, 10, 2, 10, Seq()) + sessionCatalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) + assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + // If database is not specified, temp table should be renamed first + sessionCatalog.renameTable("db_not_read", TableIdentifier("tbl1"), TableIdentifier("tbl3")) + assert(sessionCatalog.getTempTable("tbl1") == None) + assert(sessionCatalog.getTempTable("tbl3") == Some(tempTable)) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + // If database is specified, temp tables are never renamed + sessionCatalog.renameTable( + "db_not_read", TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4", Some("db2"))) + assert(sessionCatalog.getTempTable("tbl3") == Some(tempTable)) + assert(sessionCatalog.getTempTable("tbl4") == None) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) + } + + test("alter table") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val tbl1 = externalCatalog.getTable("db2", "tbl1") + sessionCatalog.alterTable("db2", tbl1.copy(properties = Map("toh" -> "frem"))) + val newTbl1 = externalCatalog.getTable("db2", "tbl1") + assert(!tbl1.properties.contains("toh")) + assert(newTbl1.properties.size == tbl1.properties.size + 1) + assert(newTbl1.properties.get("toh") == Some("frem")) + } + + test("alter table when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.alterTable("unknown_db", newTable("tbl1", "unknown_db")) + } + intercept[AnalysisException] { + catalog.alterTable("db2", newTable("unknown_table", "db2")) + } + } + + test("get table") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(sessionCatalog.getTable("db2", TableIdentifier("tbl1")) + == externalCatalog.getTable("db2", "tbl1")) + assert(sessionCatalog.getTable("db_not_read", TableIdentifier("tbl1", Some("db2"))) + == externalCatalog.getTable("db2", "tbl1")) + } + + test("get table when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.getTable("unknown_db", TableIdentifier("unknown_table")) + } + intercept[AnalysisException] { + catalog.getTable("db2", TableIdentifier("unknown_table")) + } + } + + test("lookup table relation") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val tempTable1 = Range(1, 10, 1, 10, Seq()) + sessionCatalog.createTempTable("tbl1", tempTable1, ignoreIfExists = false) + val metastoreTable1 = externalCatalog.getTable("db2", "tbl1") + // If we explicitly specify the database, we'll look up the relation in that database + assert(sessionCatalog.lookupRelation("db_not_read", TableIdentifier("tbl1", Some("db2"))) + == SubqueryAlias("tbl1", CatalogRelation("db2", metastoreTable1))) + // Otherwise, we'll first look up a temporary table with the same name + assert(sessionCatalog.lookupRelation("db2", TableIdentifier("tbl1")) + == SubqueryAlias("tbl1", tempTable1)) + // Then, if that does not exist, look up the relation in the current database + sessionCatalog.dropTable("db_not_read", TableIdentifier("tbl1"), ignoreIfNotExists = false) + assert(sessionCatalog.lookupRelation("db2", TableIdentifier("tbl1")) + == SubqueryAlias("tbl1", CatalogRelation("db2", metastoreTable1))) + } + + test("lookup table relation with alias") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val alias = "monster" + val table1 = externalCatalog.getTable("db2", "tbl1") + val withoutAlias = SubqueryAlias("tbl1", CatalogRelation("db2", table1)) + val withAlias = + SubqueryAlias(alias, + SubqueryAlias("tbl1", + CatalogRelation("db2", table1, Some(alias)))) + assert(sessionCatalog.lookupRelation( + "db_not_read", TableIdentifier("tbl1", Some("db2")), alias = None) == withoutAlias) + assert(sessionCatalog.lookupRelation( + "db_not_read", TableIdentifier("tbl1", Some("db2")), alias = Some(alias)) == withAlias) + } + + test("list tables without pattern") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempTable = Range(1, 10, 2, 10, Seq()) + catalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) + catalog.createTempTable("tbl4", tempTable, ignoreIfExists = false) + assert(catalog.listTables("db1").toSet == + Set(TableIdentifier("tbl1"), TableIdentifier("tbl4"))) + assert(catalog.listTables("db2").toSet == + Set(TableIdentifier("tbl1"), + TableIdentifier("tbl4"), + TableIdentifier("tbl1", Some("db2")), + TableIdentifier("tbl2", Some("db2")))) + intercept[AnalysisException] { catalog.listTables("unknown_db") } + } + test("list tables with pattern") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempTable = Range(1, 10, 2, 10, Seq()) + catalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) + catalog.createTempTable("tbl4", tempTable, ignoreIfExists = false) + assert(catalog.listTables("db1", "*").toSet == + Set(TableIdentifier("tbl1"), TableIdentifier("tbl4"))) + assert(catalog.listTables("db2", "*").toSet == + Set(TableIdentifier("tbl1"), + TableIdentifier("tbl4"), + TableIdentifier("tbl1", Some("db2")), + TableIdentifier("tbl2", Some("db2")))) + assert(catalog.listTables("db2", "tbl*").toSet == + Set(TableIdentifier("tbl1"), + TableIdentifier("tbl4"), + TableIdentifier("tbl1", Some("db2")), + TableIdentifier("tbl2", Some("db2")))) + assert(catalog.listTables("db2", "*1").toSet == + Set(TableIdentifier("tbl1"), TableIdentifier("tbl1", Some("db2")))) + intercept[AnalysisException] { catalog.listTables("unknown_db") } } } From 3da16fb3473b750f13ffcbbb8aaf9a7de7292897 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 19:04:03 -0700 Subject: [PATCH 15/20] Implement tests for table partitions --- .../catalyst/catalog/CatalogTestCases.scala | 2 +- .../catalog/SessionCatalogSuite.scala | 158 ++++++++++++++++++ 2 files changed, 159 insertions(+), 1 deletion(-) 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 2b12504bab86..e89dea4b7f32 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 @@ -342,7 +342,7 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { test("alter partitions") { val catalog = newBasicCatalog() - try{ + try { // Note: Before altering table partitions in Hive, you *must* set the current database // to the one that contains the table of interest. Otherwise you will end up with the // most helpful error message ever: "Unable to alter partition. alter is not possible." 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 51b35b2a272c..c331da0dc351 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 @@ -370,4 +370,162 @@ class SessionCatalogSuite extends SparkFunSuite { intercept[AnalysisException] { catalog.listTables("unknown_db") } } + // -------------------------------------------------------------------------- + // Partitions + // -------------------------------------------------------------------------- + + test("basic create and list partitions") { + val externalCatalog = newEmptyCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + sessionCatalog.createDatabase(newDb("mydb"), ignoreIfExists = false) + sessionCatalog.createTable("mydb", newTable("tbl", "mydb"), ignoreIfExists = false) + sessionCatalog.createPartitions( + "mydb", TableIdentifier("tbl"), Seq(part1, part2), ignoreIfExists = false) + assert(catalogPartitionsEqual(externalCatalog, "mydb", "tbl", Seq(part1, part2))) + } + + test("create partitions when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.createPartitions( + "does_not_exist", TableIdentifier("tbl1"), Seq(), ignoreIfExists = false) + } + intercept[AnalysisException] { + catalog.createPartitions( + "db2", TableIdentifier("does_not_exist"), Seq(), ignoreIfExists = false) + } + } + + test("create partitions that already exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.createPartitions( + "db2", TableIdentifier("tbl2"), Seq(part1), ignoreIfExists = false) + } + catalog.createPartitions("db2", TableIdentifier("tbl2"), Seq(part1), ignoreIfExists = true) + } + + test("drop partitions") { + val externalCatalog1 = newBasicCatalog() + val sessionCatalog1 = new SessionCatalog(externalCatalog1) + assert(catalogPartitionsEqual(externalCatalog1, "db2", "tbl2", Seq(part1, part2))) + sessionCatalog1.dropPartitions( + "db2", TableIdentifier("tbl2"), Seq(part1.spec), ignoreIfNotExists = false) + assert(catalogPartitionsEqual(externalCatalog1, "db2", "tbl2", Seq(part2))) + val externalCatalog2 = newBasicCatalog() + val sessionCatalog2 = new SessionCatalog(externalCatalog2) + assert(catalogPartitionsEqual(externalCatalog2, "db2", "tbl2", Seq(part1, part2))) + sessionCatalog2.dropPartitions( + "db2", TableIdentifier("tbl2"), Seq(part1.spec, part2.spec), ignoreIfNotExists = false) + assert(externalCatalog2.listPartitions("db2", "tbl2").isEmpty) + } + + test("drop partitions when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.dropPartitions( + "does_not_exist", TableIdentifier("tbl1"), Seq(), ignoreIfNotExists = false) + } + intercept[AnalysisException] { + catalog.dropPartitions( + "db2", TableIdentifier("does_not_exist"), Seq(), ignoreIfNotExists = false) + } + } + + test("drop partitions that do not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.dropPartitions( + "db2", TableIdentifier("tbl2"), Seq(part3.spec), ignoreIfNotExists = false) + } + catalog.dropPartitions( + "db2", TableIdentifier("tbl2"), Seq(part3.spec), ignoreIfNotExists = true) + } + + test("get partition") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(sessionCatalog.getPartition( + "db2", TableIdentifier("tbl2"), part1.spec).spec == part1.spec) + assert(sessionCatalog.getPartition( + "db2", TableIdentifier("tbl2"), part2.spec).spec == part2.spec) + intercept[AnalysisException] { + sessionCatalog.getPartition("db2", TableIdentifier("tbl1"), part3.spec) + } + } + + test("get partition when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.getPartition("does_not_exist", TableIdentifier("tbl1"), part1.spec) + } + intercept[AnalysisException] { + catalog.getPartition("db2", TableIdentifier("does_not_exist"), part1.spec) + } + } + + test("rename partitions") { + val catalog = new SessionCatalog(newBasicCatalog()) + val newPart1 = part1.copy(spec = Map("a" -> "100", "b" -> "101")) + val newPart2 = part2.copy(spec = Map("a" -> "200", "b" -> "201")) + val newSpecs = Seq(newPart1.spec, newPart2.spec) + catalog.renamePartitions("db2", TableIdentifier("tbl2"), Seq(part1.spec, part2.spec), newSpecs) + assert(catalog.getPartition( + "db2", TableIdentifier("tbl2"), newPart1.spec).spec === newPart1.spec) + assert(catalog.getPartition( + "db2", TableIdentifier("tbl2"), newPart2.spec).spec === newPart2.spec) + // The old partitions should no longer exist + intercept[AnalysisException] { + catalog.getPartition("db2", TableIdentifier("tbl2"), part1.spec) + } + intercept[AnalysisException] { + catalog.getPartition("db2", TableIdentifier("tbl2"), part2.spec) + } + } + + test("rename partitions when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.renamePartitions( + "does_not_exist", TableIdentifier("tbl1"), Seq(part1.spec), Seq(part2.spec)) + } + intercept[AnalysisException] { + catalog.renamePartitions( + "db2", TableIdentifier("does_not_exist"), Seq(part1.spec), Seq(part2.spec)) + } + } + + test("alter partitions") { + val catalog = new SessionCatalog(newBasicCatalog()) + val newLocation = newUriForDatabase() + // alter but keep spec the same + val oldPart1 = catalog.getPartition("db2", TableIdentifier("tbl2"), part1.spec) + val oldPart2 = catalog.getPartition("db2", TableIdentifier("tbl2"), part2.spec) + catalog.alterPartitions("db2", TableIdentifier("tbl2"), Seq( + oldPart1.copy(storage = storageFormat.copy(locationUri = Some(newLocation))), + oldPart2.copy(storage = storageFormat.copy(locationUri = Some(newLocation))))) + val newPart1 = catalog.getPartition("db2", TableIdentifier("tbl2"), part1.spec) + val newPart2 = catalog.getPartition("db2", TableIdentifier("tbl2"), part2.spec) + assert(newPart1.storage.locationUri == Some(newLocation)) + assert(newPart2.storage.locationUri == Some(newLocation)) + assert(oldPart1.storage.locationUri != Some(newLocation)) + assert(oldPart2.storage.locationUri != Some(newLocation)) + // alter but change spec, should fail because new partition specs do not exist yet + val badPart1 = part1.copy(spec = Map("a" -> "v1", "b" -> "v2")) + val badPart2 = part2.copy(spec = Map("a" -> "v3", "b" -> "v4")) + intercept[AnalysisException] { + catalog.alterPartitions("db2", TableIdentifier("tbl2"), Seq(badPart1, badPart2)) + } + } + + test("alter partitions when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.alterPartitions("does_not_exist", TableIdentifier("tbl1"), Seq(part1)) + } + intercept[AnalysisException] { + catalog.alterPartitions("db2", TableIdentifier("does_not_exist"), Seq(part1)) + } + } + } From 8c84dd803829ffcb8c82ee2f593ef58c3c5c94c9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 19:41:30 -0700 Subject: [PATCH 16/20] Implement tests for functions --- .../sql/catalyst/catalog/SessionCatalog.scala | 13 +- .../catalog/SessionCatalogSuite.scala | 194 ++++++++++++++++++ 2 files changed, 206 insertions(+), 1 deletion(-) 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 2a8f2cd21e1f..3d002278543a 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 @@ -465,7 +465,18 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { val functionsInCurrentDb = externalCatalog.listFunctions(currentDb, pattern).map { f => FunctionIdentifier(f, Some(currentDb)) } - functionsInCurrentDb ++ tempFunctions.keys.asScala.map { f => FunctionIdentifier(f) } + val regex = pattern.replaceAll("\\*", ".*").r + val _tempFunctions = tempFunctions.keys().asScala + .filter { f => regex.pattern.matcher(f).matches() } + .map { f => FunctionIdentifier(f) } + functionsInCurrentDb ++ _tempFunctions + } + + /** + * Return a temporary function. For testing only. + */ + private[catalog] def getTempFunction(name: String): Option[CatalogFunction] = { + Option(tempFunctions.get(name)) } } 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 c331da0dc351..11b7034605df 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 @@ -528,4 +528,198 @@ class SessionCatalogSuite extends SparkFunSuite { } } + // -------------------------------------------------------------------------- + // Functions + // -------------------------------------------------------------------------- + + test("basic create and list functions") { + val externalCatalog = newEmptyCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + sessionCatalog.createDatabase(newDb("mydb"), ignoreIfExists = false) + sessionCatalog.createFunction("mydb", newFunc("myfunc")) + assert(externalCatalog.listFunctions("mydb", "*").toSet == Set("myfunc")) + } + + test("create function when database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.createFunction("does_not_exist", newFunc()) + } + } + + test("create function that already exists") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.createFunction("db2", newFunc("func1")) + } + } + + test("create temp function") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempFunc1 = newFunc("temp1") + val tempFunc2 = newFunc("temp2") + catalog.createTempFunction(tempFunc1, ignoreIfExists = false) + catalog.createTempFunction(tempFunc2, ignoreIfExists = false) + assert(catalog.getTempFunction("temp1") == Some(tempFunc1)) + assert(catalog.getTempFunction("temp2") == Some(tempFunc2)) + assert(catalog.getTempFunction("temp3") == None) + // Temporary function already exists + intercept[AnalysisException] { + catalog.createTempFunction(tempFunc1, ignoreIfExists = false) + } + // Temporary function is overridden + val tempFunc3 = tempFunc1.copy(className = "something else") + catalog.createTempFunction(tempFunc3, ignoreIfExists = true) + assert(catalog.getTempFunction("temp1") == Some(tempFunc3)) + } + + test("drop function") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1")) + sessionCatalog.dropFunction("db2", FunctionIdentifier("func1")) + assert(externalCatalog.listFunctions("db2", "*").isEmpty) + } + + test("drop function when database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.dropFunction("does_not_exist", FunctionIdentifier("something")) + } + } + + test("drop function that does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.dropFunction("db2", FunctionIdentifier("does_not_exist")) + } + } + + test("drop temp function") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempFunc = newFunc("func1") + catalog.createTempFunction(tempFunc, ignoreIfExists = false) + assert(catalog.getTempFunction("func1") == Some(tempFunc)) + catalog.dropTempFunction("func1", ignoreIfNotExists = false) + assert(catalog.getTempFunction("func1") == None) + intercept[AnalysisException] { + catalog.dropTempFunction("func1", ignoreIfNotExists = false) + } + catalog.dropTempFunction("func1", ignoreIfNotExists = true) + } + + test("get function") { + val catalog = new SessionCatalog(newBasicCatalog()) + assert(catalog.getFunction("db2", FunctionIdentifier("func1")) == + CatalogFunction(FunctionIdentifier("func1", Some("db2")), funcClass)) + intercept[AnalysisException] { + catalog.getFunction("db2", FunctionIdentifier("does_not_exist")) + } + } + + test("get function when database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.getFunction("does_not_exist", FunctionIdentifier("func1")) + } + } + + test("get temp function") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val metastoreFunc = externalCatalog.getFunction("db2", "func1") + val tempFunc = newFunc("func1").copy(className = "something weird") + sessionCatalog.createTempFunction(tempFunc, ignoreIfExists = false) + // If a database is specified, we'll always return the function in that database + assert(sessionCatalog.getFunction("db2", FunctionIdentifier("func1", Some("db2"))) + == metastoreFunc) + // If no database is specified, we'll first return temporary functions + assert(sessionCatalog.getFunction("db2", FunctionIdentifier("func1")) == tempFunc) + // Then, if no such temporary function exist, check the current database + sessionCatalog.dropTempFunction("func1", ignoreIfNotExists = false) + assert(sessionCatalog.getFunction("db2", FunctionIdentifier("func1")) == metastoreFunc) + } + + test("rename function") { + val catalog = new SessionCatalog(newBasicCatalog()) + val newName = "funcky" + assert(catalog.getFunction("db2", FunctionIdentifier("func1")).className == funcClass) + catalog.renameFunction("db2", FunctionIdentifier("func1"), FunctionIdentifier(newName)) + intercept[AnalysisException] { catalog.getFunction("db2", FunctionIdentifier("func1")) } + assert(catalog.getFunction("db2", FunctionIdentifier(newName)).name.funcName == newName) + assert(catalog.getFunction("db2", FunctionIdentifier(newName)).className == funcClass) + intercept[AnalysisException] { + catalog.renameFunction("db2", FunctionIdentifier("does_not_exist"), FunctionIdentifier("x")) + } + } + + test("rename function when database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.renameFunction( + "does_not_exist", FunctionIdentifier("func1"), FunctionIdentifier("func5")) + } + } + + test("rename temp function") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val tempFunc = newFunc("func1").copy(className = "something weird") + sessionCatalog.createTempFunction(tempFunc, ignoreIfExists = false) + // If a database is specified, we'll always rename the function in that database + sessionCatalog.renameFunction( + "db2", FunctionIdentifier("func1", Some("db2")), FunctionIdentifier("func3", Some("db2"))) + assert(sessionCatalog.getTempFunction("func1") == Some(tempFunc)) + assert(sessionCatalog.getTempFunction("func3") == None) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func3")) + sessionCatalog.createFunction("db2", newFunc("func1", Some("db2"))) + // If no database is specified, we'll first rename temporary functions + sessionCatalog.renameFunction("db2", FunctionIdentifier("func1"), FunctionIdentifier("func4")) + assert(sessionCatalog.getTempFunction("func4") == + Some(tempFunc.copy(name = FunctionIdentifier("func4")))) + assert(sessionCatalog.getTempFunction("func1") == None) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1", "func3")) + // Then, if no such temporary function exist, rename the function in the current database + sessionCatalog.renameFunction("db2", FunctionIdentifier("func1"), FunctionIdentifier("func5")) + assert(sessionCatalog.getTempFunction("func5") == None) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func3", "func5")) + } + + test("alter function") { + val catalog = new SessionCatalog(newBasicCatalog()) + assert(catalog.getFunction("db2", FunctionIdentifier("func1")).className == funcClass) + catalog.alterFunction("db2", newFunc("func1").copy(className = "muhaha")) + assert(catalog.getFunction("db2", FunctionIdentifier("func1")).className == "muhaha") + intercept[AnalysisException] { catalog.alterFunction("db2", newFunc("funcky")) } + } + + test("alter function when database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + intercept[AnalysisException] { + catalog.alterFunction("does_not_exist", newFunc()) + } + } + + test("list functions") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempFunc1 = newFunc("func1").copy(className = "march") + val tempFunc2 = newFunc("yes_me").copy(className = "april") + catalog.createFunction("db2", newFunc("func2")) + catalog.createFunction("db2", newFunc("not_me")) + catalog.createTempFunction(tempFunc1, ignoreIfExists = false) + catalog.createTempFunction(tempFunc2, ignoreIfExists = false) + assert(catalog.listFunctions("db1", "*").toSet == + Set(FunctionIdentifier("func1"), FunctionIdentifier("yes_me"))) + assert(catalog.listFunctions("db2", "*").toSet == + Set(FunctionIdentifier("func1"), + FunctionIdentifier("yes_me"), + FunctionIdentifier("func1", Some("db2")), + FunctionIdentifier("func2", Some("db2")), + FunctionIdentifier("not_me", Some("db2")))) + assert(catalog.listFunctions("db2", "func*").toSet == + Set(FunctionIdentifier("func1"), + FunctionIdentifier("func1", Some("db2")), + FunctionIdentifier("func2", Some("db2")))) + } + } From ff1c2c4661986622e8071a39922e25033b3e62ab Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 15 Mar 2016 19:42:22 -0700 Subject: [PATCH 17/20] Add TODO --- .../org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 2 ++ 1 file changed, 2 insertions(+) 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 3d002278543a..61437a9d6346 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 @@ -458,6 +458,8 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { } } + // TODO: implement lookupFunction that returns something from the registry itself + /** * List all matching functions in the current database, including temporary functions. */ From 6d9fa2f946ac93ebc95a9f25cf515fb0ea54b17c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 16 Mar 2016 12:31:52 -0700 Subject: [PATCH 18/20] Keep track of current database in SessionCatalog This allows us to not pass it into every single method like we used to before this commit. --- .../sql/catalyst/catalog/SessionCatalog.scala | 106 ++++---- .../catalog/SessionCatalogSuite.scala | 239 ++++++++++-------- 2 files changed, 173 insertions(+), 172 deletions(-) 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 61437a9d6346..cd8a08364047 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 @@ -35,9 +35,14 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { import ExternalCatalog._ 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. + private[this] var currentDb = "default" + // ---------------------------------------------------------------------------- // Databases // ---------------------------------------------------------------------------- @@ -72,6 +77,12 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { externalCatalog.listDatabases(pattern) } + def getCurrentDatabase: String = currentDb + + def setCurrentDatabase(db: String): Unit = { + currentDb = db + } + // ---------------------------------------------------------------------------- // Tables // ---------------------------------------------------------------------------- @@ -89,10 +100,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Create a metastore table in the database specified in `tableDefinition`. * If no such database is specified, create it in the current database. */ - def createTable( - currentDb: String, - tableDefinition: CatalogTable, - ignoreIfExists: Boolean): Unit = { + def createTable(tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = { val db = tableDefinition.name.database.getOrElse(currentDb) val newTableDefinition = tableDefinition.copy( name = TableIdentifier(tableDefinition.name.table, Some(db))) @@ -108,7 +116,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterTable(currentDb: String, tableDefinition: CatalogTable): Unit = { + def alterTable(tableDefinition: CatalogTable): Unit = { val db = tableDefinition.name.database.getOrElse(currentDb) val newTableDefinition = tableDefinition.copy( name = TableIdentifier(tableDefinition.name.table, Some(db))) @@ -119,7 +127,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Retrieve the metadata of an existing metastore table. * If no database is specified, assume the table is in the current database. */ - def getTable(currentDb: String, name: TableIdentifier): CatalogTable = { + def getTable(name: TableIdentifier): CatalogTable = { val db = name.database.getOrElse(currentDb) externalCatalog.getTable(db, name.table) } @@ -150,10 +158,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * * This assumes the database specified in `oldName` matches the one specified in `newName`. */ - def renameTable( - currentDb: String, - oldName: TableIdentifier, - newName: TableIdentifier): Unit = { + def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = { if (oldName.database != newName.database) { throw new AnalysisException("rename does not support moving tables across databases") } @@ -173,10 +178,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * If no database is specified, this will first attempt to drop a temporary table with * the same name, then, if that does not exist, drop the table from the current database. */ - def dropTable( - currentDb: String, - name: TableIdentifier, - ignoreIfNotExists: Boolean): Unit = { + def dropTable(name: TableIdentifier, ignoreIfNotExists: Boolean): Unit = { val db = name.database.getOrElse(currentDb) if (name.database.isDefined || !tempTables.containsKey(name.table)) { externalCatalog.dropTable(db, name.table, ignoreIfNotExists) @@ -192,10 +194,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * If no database is specified, this will first attempt to return a temporary table with * the same name, then, if that does not exist, return the table from the current database. */ - def lookupRelation( - currentDb: String, - name: TableIdentifier, - alias: Option[String] = None): LogicalPlan = { + def lookupRelation(name: TableIdentifier, alias: Option[String] = None): LogicalPlan = { val db = name.database.getOrElse(currentDb) val relation = if (name.database.isDefined || !tempTables.containsKey(name.table)) { @@ -211,28 +210,25 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { } /** - * List all tables in the current database, including temporary tables. + * List all tables in the specified database, including temporary tables. */ - def listTables(currentDb: String): Seq[TableIdentifier] = { - val tablesInCurrentDb = externalCatalog.listTables(currentDb).map { t => - TableIdentifier(t, Some(currentDb)) - } + 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) } - tablesInCurrentDb ++ _tempTables + dbTables ++ _tempTables } /** - * List all matching tables in the current database, including temporary tables. + * List all matching tables in the specified database, including temporary tables. */ - def listTables(currentDb: String, pattern: String): Seq[TableIdentifier] = { - val tablesInCurrentDb = externalCatalog.listTables(currentDb, pattern).map { t => - TableIdentifier(t, Some(currentDb)) - } + def listTables(db: String, pattern: String): Seq[TableIdentifier] = { + val dbTables = + externalCatalog.listTables(db, pattern).map { t => TableIdentifier(t, Some(currentDb)) } val regex = pattern.replaceAll("\\*", ".*").r val _tempTables = tempTables.keys().asScala .filter { t => regex.pattern.matcher(t).matches() } .map { t => TableIdentifier(t) } - tablesInCurrentDb ++ _tempTables + dbTables ++ _tempTables } /** @@ -260,7 +256,6 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * If no database is specified, assume the table is in the current database. */ def createPartitions( - currentDb: String, tableName: TableIdentifier, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { @@ -273,7 +268,6 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * If no database is specified, assume the table is in the current database. */ def dropPartitions( - currentDb: String, tableName: TableIdentifier, parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = { @@ -288,7 +282,6 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * If no database is specified, assume the table is in the current database. */ def renamePartitions( - currentDb: String, tableName: TableIdentifier, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { @@ -305,10 +298,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterPartitions( - currentDb: String, - tableName: TableIdentifier, - parts: Seq[CatalogTablePartition]): Unit = { + def alterPartitions(tableName: TableIdentifier, parts: Seq[CatalogTablePartition]): Unit = { val db = tableName.database.getOrElse(currentDb) externalCatalog.alterPartitions(db, tableName.table, parts) } @@ -317,10 +307,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Retrieve the metadata of a table partition, assuming it exists. * If no database is specified, assume the table is in the current database. */ - def getPartition( - currentDb: String, - tableName: TableIdentifier, - spec: TablePartitionSpec): CatalogTablePartition = { + def getPartition(tableName: TableIdentifier, spec: TablePartitionSpec): CatalogTablePartition = { val db = tableName.database.getOrElse(currentDb) externalCatalog.getPartition(db, tableName.table, spec) } @@ -329,9 +316,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * List all partitions in a table, assuming it exists. * If no database is specified, assume the table is in the current database. */ - def listPartitions( - currentDb: String, - tableName: TableIdentifier): Seq[CatalogTablePartition] = { + def listPartitions(tableName: TableIdentifier): Seq[CatalogTablePartition] = { val db = tableName.database.getOrElse(currentDb) externalCatalog.listPartitions(db, tableName.table) } @@ -353,7 +338,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Create a metastore function in the database specified in `funcDefinition`. * If no such database is specified, create it in the current database. */ - def createFunction(currentDb: String, funcDefinition: CatalogFunction): Unit = { + def createFunction(funcDefinition: CatalogFunction): Unit = { val db = funcDefinition.name.database.getOrElse(currentDb) val newFuncDefinition = funcDefinition.copy( name = FunctionIdentifier(funcDefinition.name.funcName, Some(db))) @@ -364,7 +349,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Drop a metastore function. * If no database is specified, assume the function is in the current database. */ - def dropFunction(currentDb: String, name: FunctionIdentifier): Unit = { + def dropFunction(name: FunctionIdentifier): Unit = { val db = name.database.getOrElse(currentDb) externalCatalog.dropFunction(db, name.funcName) } @@ -378,7 +363,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterFunction(currentDb: String, funcDefinition: CatalogFunction): Unit = { + def alterFunction(funcDefinition: CatalogFunction): Unit = { val db = funcDefinition.name.database.getOrElse(currentDb) val newFuncDefinition = funcDefinition.copy( name = FunctionIdentifier(funcDefinition.name.funcName, Some(db))) @@ -393,9 +378,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * Create a temporary function. * This assumes no database is specified in `funcDefinition`. */ - def createTempFunction( - funcDefinition: CatalogFunction, - ignoreIfExists: Boolean): Unit = { + def createTempFunction(funcDefinition: CatalogFunction, ignoreIfExists: Boolean): Unit = { require(funcDefinition.name.database.isEmpty, "attempted to create a temporary function while specifying a database") val name = funcDefinition.name.funcName @@ -428,10 +411,10 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * * This assumes the database specified in `oldName` matches the one specified in `newName`. */ - def renameFunction( - currentDb: String, - oldName: FunctionIdentifier, - newName: FunctionIdentifier): Unit = { + def renameFunction(oldName: FunctionIdentifier, newName: FunctionIdentifier): Unit = { + if (oldName.database != newName.database) { + throw new AnalysisException("rename does not support moving functions across databases") + } val db = oldName.database.getOrElse(currentDb) if (oldName.database.isDefined || !tempFunctions.containsKey(oldName.funcName)) { externalCatalog.renameFunction(db, oldName.funcName, newName.funcName) @@ -449,7 +432,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { * If no database is specified, this will first attempt to return a temporary function with * the same name, then, if that does not exist, return the function in the current database. */ - def getFunction(currentDb: String, name: FunctionIdentifier): CatalogFunction = { + def getFunction(name: FunctionIdentifier): CatalogFunction = { val db = name.database.getOrElse(currentDb) if (name.database.isDefined || !tempFunctions.containsKey(name.funcName)) { externalCatalog.getFunction(db, name.funcName) @@ -461,17 +444,16 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { // TODO: implement lookupFunction that returns something from the registry itself /** - * List all matching functions in the current database, including temporary functions. + * List all matching functions in the specified database, including temporary functions. */ - def listFunctions(currentDb: String, pattern: String): Seq[FunctionIdentifier] = { - val functionsInCurrentDb = externalCatalog.listFunctions(currentDb, pattern).map { f => - FunctionIdentifier(f, Some(currentDb)) - } + def listFunctions(db: String, pattern: String): Seq[FunctionIdentifier] = { + val dbFunctions = + externalCatalog.listFunctions(db, pattern).map { f => FunctionIdentifier(f, Some(db)) } val regex = pattern.replaceAll("\\*", ".*").r val _tempFunctions = tempFunctions.keys().asScala .filter { f => regex.pattern.matcher(f).matches() } .map { f => FunctionIdentifier(f) } - functionsInCurrentDb ++ _tempFunctions + dbFunctions ++ _tempFunctions } /** 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 11b7034605df..2cf751203a91 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 @@ -168,10 +168,9 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) - sessionCatalog.dropTable("db2", TableIdentifier("tbl1"), ignoreIfNotExists = false) + sessionCatalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false) assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) - sessionCatalog.dropTable( - "db_not_read", TableIdentifier("tbl2", Some("db2")), ignoreIfNotExists = false) + sessionCatalog.dropTable(TableIdentifier("tbl2", Some("db2")), ignoreIfNotExists = false) assert(externalCatalog.listTables("db2").isEmpty) } @@ -179,17 +178,19 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) // Should always throw exception when the database does not exist intercept[AnalysisException] { - catalog.dropTable("unknown_db", TableIdentifier("unknown_table"), ignoreIfNotExists = false) + catalog.dropTable( + TableIdentifier("unknown_table", Some("unknown_db")), ignoreIfNotExists = false) } intercept[AnalysisException] { - catalog.dropTable("unknown_db", TableIdentifier("unknown_table"), ignoreIfNotExists = true) + catalog.dropTable( + TableIdentifier("unknown_table", Some("unknown_db")), ignoreIfNotExists = true) } // Should throw exception when the table does not exist, if ignoreIfNotExists is false intercept[AnalysisException] { - catalog.dropTable("db2", TableIdentifier("unknown_table"), ignoreIfNotExists = false) + catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false) } - catalog.dropTable("db2", TableIdentifier("unknown_table"), ignoreIfNotExists = true) + catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = true) } test("drop temp table") { @@ -197,16 +198,20 @@ class SessionCatalogSuite extends SparkFunSuite { val sessionCatalog = new SessionCatalog(externalCatalog) val tempTable = Range(1, 10, 2, 10, Seq()) sessionCatalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) + sessionCatalog.setCurrentDatabase("db2") assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be dropped first - sessionCatalog.dropTable("db_not_read", TableIdentifier("tbl1"), ignoreIfNotExists = false) + sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false) assert(sessionCatalog.getTempTable("tbl1") == None) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + // If temp table does not exist, the table in the current database should be dropped + sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false) + assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) // If database is specified, temp tables are never dropped sessionCatalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) - sessionCatalog.dropTable( - "db_not_read", TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false) + sessionCatalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false) + sessionCatalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false) assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) } @@ -215,27 +220,28 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) - sessionCatalog.renameTable("db2", TableIdentifier("tbl1"), TableIdentifier("tblone")) + sessionCatalog.renameTable( + TableIdentifier("tbl1", Some("db2")), TableIdentifier("tblone", Some("db2"))) assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbl2")) sessionCatalog.renameTable( - "db_not_read", TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo", Some("db2"))) + TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo", Some("db2"))) assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbltwo")) + // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match + intercept[AnalysisException] { + sessionCatalog.renameTable( + TableIdentifier("tblone", Some("db2")), TableIdentifier("tblones", Some("db1"))) + } } test("rename table when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.renameTable( - "unknown_db", TableIdentifier("unknown_table"), TableIdentifier("unknown_table")) + TableIdentifier("tbl1", Some("unknown_db")), TableIdentifier("tbl2", Some("unknown_db"))) } intercept[AnalysisException] { catalog.renameTable( - "db2", TableIdentifier("unknown_table"), TableIdentifier("unknown_table")) - } - // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match - intercept[AnalysisException] { - catalog.renameTable( - "db_not_read", TableIdentifier("tblone", Some("db2")), TableIdentifier("x", Some("db1"))) + TableIdentifier("unknown_table", Some("db2")), TableIdentifier("tbl2", Some("db2"))) } } @@ -244,16 +250,17 @@ class SessionCatalogSuite extends SparkFunSuite { val sessionCatalog = new SessionCatalog(externalCatalog) val tempTable = Range(1, 10, 2, 10, Seq()) sessionCatalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) + sessionCatalog.setCurrentDatabase("db2") assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be renamed first - sessionCatalog.renameTable("db_not_read", TableIdentifier("tbl1"), TableIdentifier("tbl3")) + sessionCatalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3")) assert(sessionCatalog.getTempTable("tbl1") == None) assert(sessionCatalog.getTempTable("tbl3") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is specified, temp tables are never renamed sessionCatalog.renameTable( - "db_not_read", TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4", Some("db2"))) + TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4", Some("db2"))) assert(sessionCatalog.getTempTable("tbl3") == Some(tempTable)) assert(sessionCatalog.getTempTable("tbl4") == None) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) @@ -263,7 +270,7 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) val tbl1 = externalCatalog.getTable("db2", "tbl1") - sessionCatalog.alterTable("db2", tbl1.copy(properties = Map("toh" -> "frem"))) + sessionCatalog.alterTable(tbl1.copy(properties = Map("toh" -> "frem"))) val newTbl1 = externalCatalog.getTable("db2", "tbl1") assert(!tbl1.properties.contains("toh")) assert(newTbl1.properties.size == tbl1.properties.size + 1) @@ -273,29 +280,30 @@ class SessionCatalogSuite extends SparkFunSuite { test("alter table when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.alterTable("unknown_db", newTable("tbl1", "unknown_db")) + catalog.alterTable(newTable("tbl1", "unknown_db")) } intercept[AnalysisException] { - catalog.alterTable("db2", newTable("unknown_table", "db2")) + catalog.alterTable(newTable("unknown_table", "db2")) } } test("get table") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - assert(sessionCatalog.getTable("db2", TableIdentifier("tbl1")) + sessionCatalog.setCurrentDatabase("db2") + assert(sessionCatalog.getTable(TableIdentifier("tbl1")) == externalCatalog.getTable("db2", "tbl1")) - assert(sessionCatalog.getTable("db_not_read", TableIdentifier("tbl1", Some("db2"))) + assert(sessionCatalog.getTable(TableIdentifier("tbl1", Some("db2"))) == externalCatalog.getTable("db2", "tbl1")) } test("get table when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.getTable("unknown_db", TableIdentifier("unknown_table")) + catalog.getTable(TableIdentifier("tbl1", Some("unknown_db"))) } intercept[AnalysisException] { - catalog.getTable("db2", TableIdentifier("unknown_table")) + catalog.getTable(TableIdentifier("unknown_table", Some("db2"))) } } @@ -304,16 +312,17 @@ class SessionCatalogSuite extends SparkFunSuite { val sessionCatalog = new SessionCatalog(externalCatalog) val tempTable1 = Range(1, 10, 1, 10, Seq()) sessionCatalog.createTempTable("tbl1", tempTable1, ignoreIfExists = false) + sessionCatalog.setCurrentDatabase("db2") val metastoreTable1 = externalCatalog.getTable("db2", "tbl1") // If we explicitly specify the database, we'll look up the relation in that database - assert(sessionCatalog.lookupRelation("db_not_read", TableIdentifier("tbl1", Some("db2"))) + assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1", Some("db2"))) == SubqueryAlias("tbl1", CatalogRelation("db2", metastoreTable1))) // Otherwise, we'll first look up a temporary table with the same name - assert(sessionCatalog.lookupRelation("db2", TableIdentifier("tbl1")) + assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1")) == SubqueryAlias("tbl1", tempTable1)) // Then, if that does not exist, look up the relation in the current database - sessionCatalog.dropTable("db_not_read", TableIdentifier("tbl1"), ignoreIfNotExists = false) - assert(sessionCatalog.lookupRelation("db2", TableIdentifier("tbl1")) + sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false) + assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1")) == SubqueryAlias("tbl1", CatalogRelation("db2", metastoreTable1))) } @@ -328,9 +337,9 @@ class SessionCatalogSuite extends SparkFunSuite { SubqueryAlias("tbl1", CatalogRelation("db2", table1, Some(alias)))) assert(sessionCatalog.lookupRelation( - "db_not_read", TableIdentifier("tbl1", Some("db2")), alias = None) == withoutAlias) + TableIdentifier("tbl1", Some("db2")), alias = None) == withoutAlias) assert(sessionCatalog.lookupRelation( - "db_not_read", TableIdentifier("tbl1", Some("db2")), alias = Some(alias)) == withAlias) + TableIdentifier("tbl1", Some("db2")), alias = Some(alias)) == withAlias) } test("list tables without pattern") { @@ -353,6 +362,7 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable = Range(1, 10, 2, 10, Seq()) catalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) catalog.createTempTable("tbl4", tempTable, ignoreIfExists = false) + catalog.setCurrentDatabase("db2") assert(catalog.listTables("db1", "*").toSet == Set(TableIdentifier("tbl1"), TableIdentifier("tbl4"))) assert(catalog.listTables("db2", "*").toSet == @@ -378,9 +388,9 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newEmptyCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) sessionCatalog.createDatabase(newDb("mydb"), ignoreIfExists = false) - sessionCatalog.createTable("mydb", newTable("tbl", "mydb"), ignoreIfExists = false) + sessionCatalog.createTable(newTable("tbl", "mydb"), ignoreIfExists = false) sessionCatalog.createPartitions( - "mydb", TableIdentifier("tbl"), Seq(part1, part2), ignoreIfExists = false) + TableIdentifier("tbl", Some("mydb")), Seq(part1, part2), ignoreIfExists = false) assert(catalogPartitionsEqual(externalCatalog, "mydb", "tbl", Seq(part1, part2))) } @@ -388,11 +398,11 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.createPartitions( - "does_not_exist", TableIdentifier("tbl1"), Seq(), ignoreIfExists = false) + TableIdentifier("tbl1", Some("does_not_exist")), Seq(), ignoreIfExists = false) } intercept[AnalysisException] { catalog.createPartitions( - "db2", TableIdentifier("does_not_exist"), Seq(), ignoreIfExists = false) + TableIdentifier("does_not_exist", Some("db2")), Seq(), ignoreIfExists = false) } } @@ -400,9 +410,10 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.createPartitions( - "db2", TableIdentifier("tbl2"), Seq(part1), ignoreIfExists = false) + TableIdentifier("tbl2", Some("db2")), Seq(part1), ignoreIfExists = false) } - catalog.createPartitions("db2", TableIdentifier("tbl2"), Seq(part1), ignoreIfExists = true) + catalog.createPartitions( + TableIdentifier("tbl2", Some("db2")), Seq(part1), ignoreIfExists = true) } test("drop partitions") { @@ -410,13 +421,13 @@ class SessionCatalogSuite extends SparkFunSuite { val sessionCatalog1 = new SessionCatalog(externalCatalog1) assert(catalogPartitionsEqual(externalCatalog1, "db2", "tbl2", Seq(part1, part2))) sessionCatalog1.dropPartitions( - "db2", TableIdentifier("tbl2"), Seq(part1.spec), ignoreIfNotExists = false) + TableIdentifier("tbl2", Some("db2")), Seq(part1.spec), ignoreIfNotExists = false) assert(catalogPartitionsEqual(externalCatalog1, "db2", "tbl2", Seq(part2))) val externalCatalog2 = newBasicCatalog() val sessionCatalog2 = new SessionCatalog(externalCatalog2) assert(catalogPartitionsEqual(externalCatalog2, "db2", "tbl2", Seq(part1, part2))) sessionCatalog2.dropPartitions( - "db2", TableIdentifier("tbl2"), Seq(part1.spec, part2.spec), ignoreIfNotExists = false) + TableIdentifier("tbl2", Some("db2")), Seq(part1.spec, part2.spec), ignoreIfNotExists = false) assert(externalCatalog2.listPartitions("db2", "tbl2").isEmpty) } @@ -424,11 +435,11 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.dropPartitions( - "does_not_exist", TableIdentifier("tbl1"), Seq(), ignoreIfNotExists = false) + TableIdentifier("tbl1", Some("does_not_exist")), Seq(), ignoreIfNotExists = false) } intercept[AnalysisException] { catalog.dropPartitions( - "db2", TableIdentifier("does_not_exist"), Seq(), ignoreIfNotExists = false) + TableIdentifier("does_not_exist", Some("db2")), Seq(), ignoreIfNotExists = false) } } @@ -436,76 +447,70 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.dropPartitions( - "db2", TableIdentifier("tbl2"), Seq(part3.spec), ignoreIfNotExists = false) + TableIdentifier("tbl2", Some("db2")), Seq(part3.spec), ignoreIfNotExists = false) } catalog.dropPartitions( - "db2", TableIdentifier("tbl2"), Seq(part3.spec), ignoreIfNotExists = true) + TableIdentifier("tbl2", Some("db2")), Seq(part3.spec), ignoreIfNotExists = true) } test("get partition") { - val externalCatalog = newBasicCatalog() - val sessionCatalog = new SessionCatalog(externalCatalog) - assert(sessionCatalog.getPartition( - "db2", TableIdentifier("tbl2"), part1.spec).spec == part1.spec) - assert(sessionCatalog.getPartition( - "db2", TableIdentifier("tbl2"), part2.spec).spec == part2.spec) + val catalog = new SessionCatalog(newBasicCatalog()) + catalog.setCurrentDatabase("db2") + assert(catalog.getPartition(TableIdentifier("tbl2"), part1.spec).spec == part1.spec) + assert(catalog.getPartition(TableIdentifier("tbl2"), part2.spec).spec == part2.spec) intercept[AnalysisException] { - sessionCatalog.getPartition("db2", TableIdentifier("tbl1"), part3.spec) + catalog.getPartition(TableIdentifier("tbl1"), part3.spec) } } test("get partition when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.getPartition("does_not_exist", TableIdentifier("tbl1"), part1.spec) + catalog.getPartition(TableIdentifier("tbl1", Some("does_not_exist")), part1.spec) } intercept[AnalysisException] { - catalog.getPartition("db2", TableIdentifier("does_not_exist"), part1.spec) + catalog.getPartition(TableIdentifier("does_not_exist", Some("db2")), part1.spec) } } test("rename partitions") { val catalog = new SessionCatalog(newBasicCatalog()) + catalog.setCurrentDatabase("db2") val newPart1 = part1.copy(spec = Map("a" -> "100", "b" -> "101")) val newPart2 = part2.copy(spec = Map("a" -> "200", "b" -> "201")) val newSpecs = Seq(newPart1.spec, newPart2.spec) - catalog.renamePartitions("db2", TableIdentifier("tbl2"), Seq(part1.spec, part2.spec), newSpecs) - assert(catalog.getPartition( - "db2", TableIdentifier("tbl2"), newPart1.spec).spec === newPart1.spec) - assert(catalog.getPartition( - "db2", TableIdentifier("tbl2"), newPart2.spec).spec === newPart2.spec) + catalog.renamePartitions(TableIdentifier("tbl2"), Seq(part1.spec, part2.spec), newSpecs) + assert(catalog.getPartition(TableIdentifier("tbl2"), newPart1.spec).spec === newPart1.spec) + assert(catalog.getPartition(TableIdentifier("tbl2"), newPart2.spec).spec === newPart2.spec) // The old partitions should no longer exist - intercept[AnalysisException] { - catalog.getPartition("db2", TableIdentifier("tbl2"), part1.spec) - } - intercept[AnalysisException] { - catalog.getPartition("db2", TableIdentifier("tbl2"), part2.spec) - } + intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl2"), part1.spec) } + intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl2"), part2.spec) } } test("rename partitions when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.renamePartitions( - "does_not_exist", TableIdentifier("tbl1"), Seq(part1.spec), Seq(part2.spec)) + TableIdentifier("tbl1", Some("does_not_exist")), Seq(part1.spec), Seq(part2.spec)) } intercept[AnalysisException] { catalog.renamePartitions( - "db2", TableIdentifier("does_not_exist"), Seq(part1.spec), Seq(part2.spec)) + TableIdentifier("does_not_exist", Some("db2")), Seq(part1.spec), Seq(part2.spec)) } } test("alter partitions") { val catalog = new SessionCatalog(newBasicCatalog()) + catalog.setCurrentDatabase("db2") val newLocation = newUriForDatabase() // alter but keep spec the same - val oldPart1 = catalog.getPartition("db2", TableIdentifier("tbl2"), part1.spec) - val oldPart2 = catalog.getPartition("db2", TableIdentifier("tbl2"), part2.spec) - catalog.alterPartitions("db2", TableIdentifier("tbl2"), Seq( + val oldPart1 = catalog.getPartition(TableIdentifier("tbl2"), part1.spec) + val oldPart2 = catalog.getPartition(TableIdentifier("tbl2"), part2.spec) + catalog.alterPartitions(TableIdentifier("tbl2"), Seq( oldPart1.copy(storage = storageFormat.copy(locationUri = Some(newLocation))), oldPart2.copy(storage = storageFormat.copy(locationUri = Some(newLocation))))) - val newPart1 = catalog.getPartition("db2", TableIdentifier("tbl2"), part1.spec) - val newPart2 = catalog.getPartition("db2", TableIdentifier("tbl2"), part2.spec) + val newPart1 = catalog.getPartition(TableIdentifier("tbl2"), part1.spec) + val newPart2 = catalog.getPartition(TableIdentifier("tbl2"), part2.spec) assert(newPart1.storage.locationUri == Some(newLocation)) assert(newPart2.storage.locationUri == Some(newLocation)) assert(oldPart1.storage.locationUri != Some(newLocation)) @@ -514,17 +519,17 @@ class SessionCatalogSuite extends SparkFunSuite { val badPart1 = part1.copy(spec = Map("a" -> "v1", "b" -> "v2")) val badPart2 = part2.copy(spec = Map("a" -> "v3", "b" -> "v4")) intercept[AnalysisException] { - catalog.alterPartitions("db2", TableIdentifier("tbl2"), Seq(badPart1, badPart2)) + catalog.alterPartitions(TableIdentifier("tbl2"), Seq(badPart1, badPart2)) } } test("alter partitions when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.alterPartitions("does_not_exist", TableIdentifier("tbl1"), Seq(part1)) + catalog.alterPartitions(TableIdentifier("tbl1", Some("does_not_exist")), Seq(part1)) } intercept[AnalysisException] { - catalog.alterPartitions("db2", TableIdentifier("does_not_exist"), Seq(part1)) + catalog.alterPartitions(TableIdentifier("does_not_exist", Some("db2")), Seq(part1)) } } @@ -536,22 +541,21 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newEmptyCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) sessionCatalog.createDatabase(newDb("mydb"), ignoreIfExists = false) - sessionCatalog.createFunction("mydb", newFunc("myfunc")) + sessionCatalog.setCurrentDatabase("mydb") + sessionCatalog.createFunction(newFunc("myfunc")) assert(externalCatalog.listFunctions("mydb", "*").toSet == Set("myfunc")) } test("create function when database does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.createFunction("does_not_exist", newFunc()) + catalog.createFunction(newFunc("func5", Some("does_not_exist"))) } } test("create function that already exists") { val catalog = new SessionCatalog(newBasicCatalog()) - intercept[AnalysisException] { - catalog.createFunction("db2", newFunc("func1")) - } + intercept[AnalysisException] { catalog.createFunction(newFunc("func1", Some("db2"))) } } test("create temp function") { @@ -577,21 +581,21 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1")) - sessionCatalog.dropFunction("db2", FunctionIdentifier("func1")) + sessionCatalog.dropFunction(FunctionIdentifier("func1", Some("db2"))) assert(externalCatalog.listFunctions("db2", "*").isEmpty) } test("drop function when database does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.dropFunction("does_not_exist", FunctionIdentifier("something")) + catalog.dropFunction(FunctionIdentifier("something", Some("does_not_exist"))) } } test("drop function that does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.dropFunction("db2", FunctionIdentifier("does_not_exist")) + catalog.dropFunction(FunctionIdentifier("does_not_exist")) } } @@ -610,17 +614,17 @@ class SessionCatalogSuite extends SparkFunSuite { test("get function") { val catalog = new SessionCatalog(newBasicCatalog()) - assert(catalog.getFunction("db2", FunctionIdentifier("func1")) == + assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))) == CatalogFunction(FunctionIdentifier("func1", Some("db2")), funcClass)) intercept[AnalysisException] { - catalog.getFunction("db2", FunctionIdentifier("does_not_exist")) + catalog.getFunction(FunctionIdentifier("does_not_exist", Some("db2"))) } } test("get function when database does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.getFunction("does_not_exist", FunctionIdentifier("func1")) + catalog.getFunction(FunctionIdentifier("func1", Some("does_not_exist"))) } } @@ -630,26 +634,39 @@ class SessionCatalogSuite extends SparkFunSuite { val metastoreFunc = externalCatalog.getFunction("db2", "func1") val tempFunc = newFunc("func1").copy(className = "something weird") sessionCatalog.createTempFunction(tempFunc, ignoreIfExists = false) + sessionCatalog.setCurrentDatabase("db2") // If a database is specified, we'll always return the function in that database - assert(sessionCatalog.getFunction("db2", FunctionIdentifier("func1", Some("db2"))) - == metastoreFunc) + assert(sessionCatalog.getFunction(FunctionIdentifier("func1", Some("db2"))) == metastoreFunc) // If no database is specified, we'll first return temporary functions - assert(sessionCatalog.getFunction("db2", FunctionIdentifier("func1")) == tempFunc) + assert(sessionCatalog.getFunction(FunctionIdentifier("func1")) == tempFunc) // Then, if no such temporary function exist, check the current database sessionCatalog.dropTempFunction("func1", ignoreIfNotExists = false) - assert(sessionCatalog.getFunction("db2", FunctionIdentifier("func1")) == metastoreFunc) + assert(sessionCatalog.getFunction(FunctionIdentifier("func1")) == metastoreFunc) } test("rename function") { - val catalog = new SessionCatalog(newBasicCatalog()) + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) val newName = "funcky" - assert(catalog.getFunction("db2", FunctionIdentifier("func1")).className == funcClass) - catalog.renameFunction("db2", FunctionIdentifier("func1"), FunctionIdentifier(newName)) - intercept[AnalysisException] { catalog.getFunction("db2", FunctionIdentifier("func1")) } - assert(catalog.getFunction("db2", FunctionIdentifier(newName)).name.funcName == newName) - assert(catalog.getFunction("db2", FunctionIdentifier(newName)).className == funcClass) + assert(sessionCatalog.getFunction(FunctionIdentifier("func1", Some("db2"))).className + == funcClass) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1")) + sessionCatalog.renameFunction( + FunctionIdentifier("func1", Some("db2")), FunctionIdentifier(newName, Some("db2"))) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set(newName)) + assert(sessionCatalog.getFunction(FunctionIdentifier(newName, Some("db2"))).name.funcName + == newName) + assert(sessionCatalog.getFunction(FunctionIdentifier(newName, Some("db2"))).className + == funcClass) + // Renaming a non-existent function should fail + intercept[AnalysisException] { + sessionCatalog.renameFunction( + FunctionIdentifier("does_not_exist", Some("db2")), FunctionIdentifier("x", Some("db2"))) + } + // Renaming "db2.func1" to "db1.func2" should fail because databases don't match intercept[AnalysisException] { - catalog.renameFunction("db2", FunctionIdentifier("does_not_exist"), FunctionIdentifier("x")) + sessionCatalog.renameFunction( + FunctionIdentifier("func1", Some("db2")), FunctionIdentifier("func2", Some("db1"))) } } @@ -657,7 +674,8 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.renameFunction( - "does_not_exist", FunctionIdentifier("func1"), FunctionIdentifier("func5")) + FunctionIdentifier("func1", Some("does_not_exist")), + FunctionIdentifier("func5", Some("does_not_exist"))) } } @@ -666,37 +684,38 @@ class SessionCatalogSuite extends SparkFunSuite { val sessionCatalog = new SessionCatalog(externalCatalog) val tempFunc = newFunc("func1").copy(className = "something weird") sessionCatalog.createTempFunction(tempFunc, ignoreIfExists = false) + sessionCatalog.setCurrentDatabase("db2") // If a database is specified, we'll always rename the function in that database sessionCatalog.renameFunction( - "db2", FunctionIdentifier("func1", Some("db2")), FunctionIdentifier("func3", Some("db2"))) + FunctionIdentifier("func1", Some("db2")), FunctionIdentifier("func3", Some("db2"))) assert(sessionCatalog.getTempFunction("func1") == Some(tempFunc)) assert(sessionCatalog.getTempFunction("func3") == None) assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func3")) - sessionCatalog.createFunction("db2", newFunc("func1", Some("db2"))) + sessionCatalog.createFunction(newFunc("func1", Some("db2"))) // If no database is specified, we'll first rename temporary functions - sessionCatalog.renameFunction("db2", FunctionIdentifier("func1"), FunctionIdentifier("func4")) + sessionCatalog.renameFunction(FunctionIdentifier("func1"), FunctionIdentifier("func4")) assert(sessionCatalog.getTempFunction("func4") == Some(tempFunc.copy(name = FunctionIdentifier("func4")))) assert(sessionCatalog.getTempFunction("func1") == None) assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1", "func3")) // Then, if no such temporary function exist, rename the function in the current database - sessionCatalog.renameFunction("db2", FunctionIdentifier("func1"), FunctionIdentifier("func5")) + sessionCatalog.renameFunction(FunctionIdentifier("func1"), FunctionIdentifier("func5")) assert(sessionCatalog.getTempFunction("func5") == None) assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func3", "func5")) } test("alter function") { val catalog = new SessionCatalog(newBasicCatalog()) - assert(catalog.getFunction("db2", FunctionIdentifier("func1")).className == funcClass) - catalog.alterFunction("db2", newFunc("func1").copy(className = "muhaha")) - assert(catalog.getFunction("db2", FunctionIdentifier("func1")).className == "muhaha") - intercept[AnalysisException] { catalog.alterFunction("db2", newFunc("funcky")) } + assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))).className == funcClass) + catalog.alterFunction(newFunc("func1", Some("db2")).copy(className = "muhaha")) + assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))).className == "muhaha") + intercept[AnalysisException] { catalog.alterFunction(newFunc("funcky", Some("db2"))) } } test("alter function when database does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { - catalog.alterFunction("does_not_exist", newFunc()) + catalog.alterFunction(newFunc("func5", Some("does_not_exist"))) } } @@ -704,8 +723,8 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) val tempFunc1 = newFunc("func1").copy(className = "march") val tempFunc2 = newFunc("yes_me").copy(className = "april") - catalog.createFunction("db2", newFunc("func2")) - catalog.createFunction("db2", newFunc("not_me")) + catalog.createFunction(newFunc("func2", Some("db2"))) + catalog.createFunction(newFunc("not_me", Some("db2"))) catalog.createTempFunction(tempFunc1, ignoreIfExists = false) catalog.createTempFunction(tempFunc2, ignoreIfExists = false) assert(catalog.listFunctions("db1", "*").toSet == From 08969cdcaf8196a30a3c879f956a8386fe400695 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 16 Mar 2016 13:21:50 -0700 Subject: [PATCH 19/20] Fix tests --- .../spark/sql/catalyst/identifiers.scala | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index 54599028888b..87f4d1b007fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -24,7 +24,8 @@ package org.apache.spark.sql.catalyst * Format (unquoted): "name" or "db.name" * Format (quoted): "`name`" or "`db`.`name`" */ -private[sql] abstract class IdentifierWithDatabase(name: String) { +sealed trait IdentifierWithDatabase { + val name: String def database: Option[String] def quotedString: String = database.map(db => s"`$db`.`$name`").getOrElse(s"`$name`") def unquotedString: String = database.map(db => s"$db.$name").getOrElse(name) @@ -36,15 +37,16 @@ private[sql] abstract class IdentifierWithDatabase(name: String) { * Identifies a table in a database. * If `database` is not defined, the current database is used. */ -private[sql] case class TableIdentifier( - table: String, - database: Option[String]) - extends IdentifierWithDatabase(table) { +case class TableIdentifier(table: String, database: Option[String]) + extends IdentifierWithDatabase { + + override val name: String = table def this(name: String) = this(name, None) + } -private[sql] object TableIdentifier { +object TableIdentifier { def apply(tableName: String): TableIdentifier = new TableIdentifier(tableName) } @@ -53,14 +55,14 @@ private[sql] object TableIdentifier { * Identifies a function in a database. * If `database` is not defined, the current database is used. */ -private[sql] case class FunctionIdentifier( - funcName: String, - database: Option[String]) - extends IdentifierWithDatabase(funcName) { +case class FunctionIdentifier(funcName: String, database: Option[String]) + extends IdentifierWithDatabase { + + override val name: String = funcName def this(name: String) = this(name, None) } -private[sql] object FunctionIdentifier { +object FunctionIdentifier { def apply(funcName: String): FunctionIdentifier = new FunctionIdentifier(funcName) } From ad43a5ffdeeb881aaed8944971b63a27d1f4257f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 16 Mar 2016 14:35:02 -0700 Subject: [PATCH 20/20] Expand test scope + clean up test code --- .../catalyst/catalog/InMemoryCatalog.scala | 27 +- .../sql/catalyst/catalog/SessionCatalog.scala | 5 +- .../catalyst/catalog/CatalogTestCases.scala | 6 +- .../catalog/SessionCatalogSuite.scala | 298 ++++++++++++------ 4 files changed, 231 insertions(+), 105 deletions(-) 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 e476b319da12..7ead1ddebe85 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 @@ -69,19 +69,20 @@ class InMemoryCatalog extends ExternalCatalog { private def requireFunctionExists(db: String, funcName: String): Unit = { if (!existsFunction(db, funcName)) { - throw new AnalysisException(s"Function $funcName does not exist in $db database") + throw new AnalysisException(s"Function '$funcName' does not exist in database '$db'") } } private def requireTableExists(db: String, table: String): Unit = { if (!existsTable(db, table)) { - throw new AnalysisException(s"Table $table does not exist in $db database") + throw new AnalysisException(s"Table '$table' does not exist in database '$db'") } } private def requirePartitionExists(db: String, table: String, spec: TablePartitionSpec): Unit = { if (!existsPartition(db, table, spec)) { - throw new AnalysisException(s"Partition does not exist in database $db table $table: $spec") + throw new AnalysisException( + s"Partition does not exist in database '$db' table '$table': '$spec'") } } @@ -94,7 +95,7 @@ class InMemoryCatalog extends ExternalCatalog { ignoreIfExists: Boolean): Unit = synchronized { if (catalog.contains(dbDefinition.name)) { if (!ignoreIfExists) { - throw new AnalysisException(s"Database ${dbDefinition.name} already exists.") + throw new AnalysisException(s"Database '${dbDefinition.name}' already exists.") } } else { catalog.put(dbDefinition.name, new DatabaseDesc(dbDefinition)) @@ -109,17 +110,17 @@ class InMemoryCatalog extends ExternalCatalog { if (!cascade) { // If cascade is false, make sure the database is empty. if (catalog(db).tables.nonEmpty) { - throw new AnalysisException(s"Database $db is not empty. One or more tables exist.") + throw new AnalysisException(s"Database '$db' is not empty. One or more tables exist.") } if (catalog(db).functions.nonEmpty) { - throw new AnalysisException(s"Database $db is not empty. One or more functions exist.") + throw new AnalysisException(s"Database '$db' is not empty. One or more functions exist.") } } // Remove the database. catalog.remove(db) } else { if (!ignoreIfNotExists) { - throw new AnalysisException(s"Database $db does not exist") + throw new AnalysisException(s"Database '$db' does not exist") } } } @@ -160,7 +161,7 @@ class InMemoryCatalog extends ExternalCatalog { val table = tableDefinition.name.table if (existsTable(db, table)) { if (!ignoreIfExists) { - throw new AnalysisException(s"Table $table already exists in $db database") + throw new AnalysisException(s"Table '$table' already exists in database '$db'") } } else { catalog(db).tables.put(table, new TableDesc(tableDefinition)) @@ -224,8 +225,8 @@ class InMemoryCatalog extends ExternalCatalog { val dupSpecs = parts.collect { case p if existingParts.contains(p.spec) => p.spec } if (dupSpecs.nonEmpty) { val dupSpecsStr = dupSpecs.mkString("\n===\n") - throw new AnalysisException( - s"The following partitions already exist in database $db table $table:\n$dupSpecsStr") + throw new AnalysisException("The following partitions already exist in database " + + s"'$db' table '$table':\n$dupSpecsStr") } } parts.foreach { p => existingParts.put(p.spec, p) } @@ -242,8 +243,8 @@ class InMemoryCatalog extends ExternalCatalog { val missingSpecs = partSpecs.collect { case s if !existingParts.contains(s) => s } if (missingSpecs.nonEmpty) { val missingSpecsStr = missingSpecs.mkString("\n===\n") - throw new AnalysisException( - s"The following partitions do not exist in database $db table $table:\n$missingSpecsStr") + throw new AnalysisException("The following partitions do not exist in database " + + s"'$db' table '$table':\n$missingSpecsStr") } } partSpecs.foreach(existingParts.remove) @@ -295,7 +296,7 @@ class InMemoryCatalog extends ExternalCatalog { override def createFunction(db: String, func: CatalogFunction): Unit = synchronized { requireDbExists(db) if (existsFunction(db, func.name.funcName)) { - throw new AnalysisException(s"Function $func already exists in $db database") + 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 cd8a08364047..4dec0429bd1f 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 @@ -80,6 +80,9 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { def getCurrentDatabase: String = currentDb def setCurrentDatabase(db: String): Unit = { + if (!databaseExists(db)) { + throw new AnalysisException(s"cannot set current database to non-existent '$db'") + } currentDb = db } @@ -223,7 +226,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog) { */ def listTables(db: String, pattern: String): Seq[TableIdentifier] = { val dbTables = - externalCatalog.listTables(db, pattern).map { t => TableIdentifier(t, Some(currentDb)) } + externalCatalog.listTables(db, pattern).map { t => TableIdentifier(t, Some(db)) } val regex = pattern.replaceAll("\\*", ".*").r val _tempTables = tempTables.keys().asScala .filter { t => regex.pattern.matcher(t).matches() } 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 e89dea4b7f32..a1ea61920dd6 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 @@ -544,9 +544,11 @@ abstract class CatalogTestUtils { CatalogDatabase(name, name + " description", newUriForDatabase(), Map.empty) } - def newTable(name: String, db: String): CatalogTable = { + def newTable(name: String, db: String): CatalogTable = newTable(name, Some(db)) + + def newTable(name: String, database: Option[String] = None): CatalogTable = { CatalogTable( - name = TableIdentifier(name, Some(db)), + name = TableIdentifier(name, database), tableType = CatalogTableType.EXTERNAL_TABLE, storage = storageFormat, schema = Seq(CatalogColumn("col1", "int"), CatalogColumn("col2", "string")), 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 2cf751203a91..e1973ee25823 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 @@ -68,7 +68,9 @@ class SessionCatalogSuite extends SparkFunSuite { test("get database should throw exception when the database does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) - intercept[AnalysisException] { catalog.getDatabase("db_that_does_not_exist") } + intercept[AnalysisException] { + catalog.getDatabase("db_that_does_not_exist") + } } test("list databases without pattern") { @@ -142,11 +144,56 @@ class SessionCatalogSuite extends SparkFunSuite { } } + test("get/set current database") { + val catalog = new SessionCatalog(newBasicCatalog()) + assert(catalog.getCurrentDatabase == "default") + catalog.setCurrentDatabase("db2") + assert(catalog.getCurrentDatabase == "db2") + intercept[AnalysisException] { + catalog.setCurrentDatabase("deebo") + } + catalog.createDatabase(newDb("deebo"), ignoreIfExists = false) + catalog.setCurrentDatabase("deebo") + assert(catalog.getCurrentDatabase == "deebo") + } + // -------------------------------------------------------------------------- // Tables // -------------------------------------------------------------------------- - test("create temporary table") { + test("create table") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(externalCatalog.listTables("db1").isEmpty) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + sessionCatalog.createTable(newTable("tbl3", "db1"), ignoreIfExists = false) + sessionCatalog.createTable(newTable("tbl3", "db2"), ignoreIfExists = false) + assert(externalCatalog.listTables("db1").toSet == Set("tbl3")) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3")) + // Create table without explicitly specifying database + sessionCatalog.setCurrentDatabase("db1") + sessionCatalog.createTable(newTable("tbl4"), ignoreIfExists = false) + assert(externalCatalog.listTables("db1").toSet == Set("tbl3", "tbl4")) + assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3")) + } + + test("create table when database does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + // Creating table in non-existent database should always fail + intercept[AnalysisException] { + catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = false) + } + intercept[AnalysisException] { + catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = true) + } + // Table already exists + intercept[AnalysisException] { + catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false) + } + catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = true) + } + + test("create temp table") { val catalog = new SessionCatalog(newBasicCatalog()) val tempTable1 = Range(1, 10, 1, 10, Seq()) val tempTable2 = Range(1, 20, 2, 10, Seq()) @@ -170,7 +217,9 @@ class SessionCatalogSuite extends SparkFunSuite { assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) sessionCatalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false) assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) - sessionCatalog.dropTable(TableIdentifier("tbl2", Some("db2")), ignoreIfNotExists = false) + // Drop table without explicitly specifying database + sessionCatalog.setCurrentDatabase("db2") + sessionCatalog.dropTable(TableIdentifier("tbl2"), ignoreIfNotExists = false) assert(externalCatalog.listTables("db2").isEmpty) } @@ -178,15 +227,12 @@ class SessionCatalogSuite extends SparkFunSuite { val catalog = new SessionCatalog(newBasicCatalog()) // Should always throw exception when the database does not exist intercept[AnalysisException] { - catalog.dropTable( - TableIdentifier("unknown_table", Some("unknown_db")), ignoreIfNotExists = false) + catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = false) } intercept[AnalysisException] { - catalog.dropTable( - TableIdentifier("unknown_table", Some("unknown_db")), ignoreIfNotExists = true) + catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = true) } - - // Should throw exception when the table does not exist, if ignoreIfNotExists is false + // Table does not exist intercept[AnalysisException] { catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false) } @@ -226,6 +272,10 @@ class SessionCatalogSuite extends SparkFunSuite { sessionCatalog.renameTable( TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo", Some("db2"))) assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbltwo")) + // Rename table without explicitly specifying database + sessionCatalog.setCurrentDatabase("db2") + sessionCatalog.renameTable(TableIdentifier("tbltwo"), TableIdentifier("table_two")) + assert(externalCatalog.listTables("db2").toSet == Set("tblone", "table_two")) // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match intercept[AnalysisException] { sessionCatalog.renameTable( @@ -275,6 +325,11 @@ class SessionCatalogSuite extends SparkFunSuite { assert(!tbl1.properties.contains("toh")) assert(newTbl1.properties.size == tbl1.properties.size + 1) assert(newTbl1.properties.get("toh") == Some("frem")) + // Alter table without explicitly specifying database + sessionCatalog.setCurrentDatabase("db2") + sessionCatalog.alterTable(tbl1.copy(name = TableIdentifier("tbl1"))) + val newestTbl1 = externalCatalog.getTable("db2", "tbl1") + assert(newestTbl1 == tbl1) } test("alter table when database/table does not exist") { @@ -290,11 +345,12 @@ class SessionCatalogSuite extends SparkFunSuite { test("get table") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) + assert(sessionCatalog.getTable(TableIdentifier("tbl1", Some("db2"))) + == externalCatalog.getTable("db2", "tbl1")) + // Get table without explicitly specifying database sessionCatalog.setCurrentDatabase("db2") assert(sessionCatalog.getTable(TableIdentifier("tbl1")) == externalCatalog.getTable("db2", "tbl1")) - assert(sessionCatalog.getTable(TableIdentifier("tbl1", Some("db2"))) - == externalCatalog.getTable("db2", "tbl1")) } test("get table when database/table does not exist") { @@ -311,9 +367,9 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) val tempTable1 = Range(1, 10, 1, 10, Seq()) + val metastoreTable1 = externalCatalog.getTable("db2", "tbl1") sessionCatalog.createTempTable("tbl1", tempTable1, ignoreIfExists = false) sessionCatalog.setCurrentDatabase("db2") - val metastoreTable1 = externalCatalog.getTable("db2", "tbl1") // If we explicitly specify the database, we'll look up the relation in that database assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1", Some("db2"))) == SubqueryAlias("tbl1", CatalogRelation("db2", metastoreTable1))) @@ -327,19 +383,18 @@ class SessionCatalogSuite extends SparkFunSuite { } test("lookup table relation with alias") { - val externalCatalog = newBasicCatalog() - val sessionCatalog = new SessionCatalog(externalCatalog) + val catalog = new SessionCatalog(newBasicCatalog()) val alias = "monster" - val table1 = externalCatalog.getTable("db2", "tbl1") - val withoutAlias = SubqueryAlias("tbl1", CatalogRelation("db2", table1)) - val withAlias = + val tableMetadata = catalog.getTable(TableIdentifier("tbl1", Some("db2"))) + val relation = SubqueryAlias("tbl1", CatalogRelation("db2", tableMetadata)) + val relationWithAlias = SubqueryAlias(alias, SubqueryAlias("tbl1", - CatalogRelation("db2", table1, Some(alias)))) - assert(sessionCatalog.lookupRelation( - TableIdentifier("tbl1", Some("db2")), alias = None) == withoutAlias) - assert(sessionCatalog.lookupRelation( - TableIdentifier("tbl1", Some("db2")), alias = Some(alias)) == withAlias) + CatalogRelation("db2", tableMetadata, Some(alias)))) + assert(catalog.lookupRelation( + TableIdentifier("tbl1", Some("db2")), alias = None) == relation) + assert(catalog.lookupRelation( + TableIdentifier("tbl1", Some("db2")), alias = Some(alias)) == relationWithAlias) } test("list tables without pattern") { @@ -354,7 +409,9 @@ class SessionCatalogSuite extends SparkFunSuite { TableIdentifier("tbl4"), TableIdentifier("tbl1", Some("db2")), TableIdentifier("tbl2", Some("db2")))) - intercept[AnalysisException] { catalog.listTables("unknown_db") } + intercept[AnalysisException] { + catalog.listTables("unknown_db") + } } test("list tables with pattern") { @@ -362,14 +419,8 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable = Range(1, 10, 2, 10, Seq()) catalog.createTempTable("tbl1", tempTable, ignoreIfExists = false) catalog.createTempTable("tbl4", tempTable, ignoreIfExists = false) - catalog.setCurrentDatabase("db2") - assert(catalog.listTables("db1", "*").toSet == - Set(TableIdentifier("tbl1"), TableIdentifier("tbl4"))) - assert(catalog.listTables("db2", "*").toSet == - Set(TableIdentifier("tbl1"), - TableIdentifier("tbl4"), - TableIdentifier("tbl1", Some("db2")), - TableIdentifier("tbl2", Some("db2")))) + assert(catalog.listTables("db1", "*").toSet == catalog.listTables("db1").toSet) + assert(catalog.listTables("db2", "*").toSet == catalog.listTables("db2").toSet) assert(catalog.listTables("db2", "tbl*").toSet == Set(TableIdentifier("tbl1"), TableIdentifier("tbl4"), @@ -377,7 +428,9 @@ class SessionCatalogSuite extends SparkFunSuite { TableIdentifier("tbl2", Some("db2")))) assert(catalog.listTables("db2", "*1").toSet == Set(TableIdentifier("tbl1"), TableIdentifier("tbl1", Some("db2")))) - intercept[AnalysisException] { catalog.listTables("unknown_db") } + intercept[AnalysisException] { + catalog.listTables("unknown_db") + } } // -------------------------------------------------------------------------- @@ -392,6 +445,10 @@ class SessionCatalogSuite extends SparkFunSuite { sessionCatalog.createPartitions( TableIdentifier("tbl", Some("mydb")), Seq(part1, part2), ignoreIfExists = false) assert(catalogPartitionsEqual(externalCatalog, "mydb", "tbl", Seq(part1, part2))) + // Create partitions without explicitly specifying database + sessionCatalog.setCurrentDatabase("mydb") + sessionCatalog.createPartitions(TableIdentifier("tbl"), Seq(part3), ignoreIfExists = false) + assert(catalogPartitionsEqual(externalCatalog, "mydb", "tbl", Seq(part1, part2, part3))) } test("create partitions when database/table does not exist") { @@ -417,18 +474,24 @@ class SessionCatalogSuite extends SparkFunSuite { } test("drop partitions") { - val externalCatalog1 = newBasicCatalog() - val sessionCatalog1 = new SessionCatalog(externalCatalog1) - assert(catalogPartitionsEqual(externalCatalog1, "db2", "tbl2", Seq(part1, part2))) - sessionCatalog1.dropPartitions( + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(catalogPartitionsEqual(externalCatalog, "db2", "tbl2", Seq(part1, part2))) + sessionCatalog.dropPartitions( TableIdentifier("tbl2", Some("db2")), Seq(part1.spec), ignoreIfNotExists = false) - assert(catalogPartitionsEqual(externalCatalog1, "db2", "tbl2", Seq(part2))) - val externalCatalog2 = newBasicCatalog() - val sessionCatalog2 = new SessionCatalog(externalCatalog2) - assert(catalogPartitionsEqual(externalCatalog2, "db2", "tbl2", Seq(part1, part2))) - sessionCatalog2.dropPartitions( + assert(catalogPartitionsEqual(externalCatalog, "db2", "tbl2", Seq(part2))) + // Drop partitions without explicitly specifying database + sessionCatalog.setCurrentDatabase("db2") + sessionCatalog.dropPartitions( + TableIdentifier("tbl2"), Seq(part2.spec), ignoreIfNotExists = false) + assert(externalCatalog.listPartitions("db2", "tbl2").isEmpty) + // Drop multiple partitions at once + sessionCatalog.createPartitions( + TableIdentifier("tbl2", Some("db2")), Seq(part1, part2), ignoreIfExists = false) + assert(catalogPartitionsEqual(externalCatalog, "db2", "tbl2", Seq(part1, part2))) + sessionCatalog.dropPartitions( TableIdentifier("tbl2", Some("db2")), Seq(part1.spec, part2.spec), ignoreIfNotExists = false) - assert(externalCatalog2.listPartitions("db2", "tbl2").isEmpty) + assert(externalCatalog.listPartitions("db2", "tbl2").isEmpty) } test("drop partitions when database/table does not exist") { @@ -455,11 +518,17 @@ class SessionCatalogSuite extends SparkFunSuite { test("get partition") { val catalog = new SessionCatalog(newBasicCatalog()) + assert(catalog.getPartition( + TableIdentifier("tbl2", Some("db2")), part1.spec).spec == part1.spec) + assert(catalog.getPartition( + TableIdentifier("tbl2", Some("db2")), part2.spec).spec == part2.spec) + // Get partition without explicitly specifying database catalog.setCurrentDatabase("db2") assert(catalog.getPartition(TableIdentifier("tbl2"), part1.spec).spec == part1.spec) assert(catalog.getPartition(TableIdentifier("tbl2"), part2.spec).spec == part2.spec) + // Get non-existent partition intercept[AnalysisException] { - catalog.getPartition(TableIdentifier("tbl1"), part3.spec) + catalog.getPartition(TableIdentifier("tbl2"), part3.spec) } } @@ -475,16 +544,32 @@ class SessionCatalogSuite extends SparkFunSuite { test("rename partitions") { val catalog = new SessionCatalog(newBasicCatalog()) - catalog.setCurrentDatabase("db2") val newPart1 = part1.copy(spec = Map("a" -> "100", "b" -> "101")) val newPart2 = part2.copy(spec = Map("a" -> "200", "b" -> "201")) val newSpecs = Seq(newPart1.spec, newPart2.spec) - catalog.renamePartitions(TableIdentifier("tbl2"), Seq(part1.spec, part2.spec), newSpecs) - assert(catalog.getPartition(TableIdentifier("tbl2"), newPart1.spec).spec === newPart1.spec) - assert(catalog.getPartition(TableIdentifier("tbl2"), newPart2.spec).spec === newPart2.spec) - // The old partitions should no longer exist - intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl2"), part1.spec) } - intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl2"), part2.spec) } + catalog.renamePartitions( + TableIdentifier("tbl2", Some("db2")), Seq(part1.spec, part2.spec), newSpecs) + assert(catalog.getPartition( + TableIdentifier("tbl2", Some("db2")), newPart1.spec).spec === newPart1.spec) + assert(catalog.getPartition( + TableIdentifier("tbl2", Some("db2")), newPart2.spec).spec === newPart2.spec) + intercept[AnalysisException] { + catalog.getPartition(TableIdentifier("tbl2", Some("db2")), part1.spec) + } + intercept[AnalysisException] { + catalog.getPartition(TableIdentifier("tbl2", Some("db2")), part2.spec) + } + // Rename partitions without explicitly specifying database + catalog.setCurrentDatabase("db2") + catalog.renamePartitions(TableIdentifier("tbl2"), newSpecs, Seq(part1.spec, part2.spec)) + assert(catalog.getPartition(TableIdentifier("tbl2"), part1.spec).spec === part1.spec) + assert(catalog.getPartition(TableIdentifier("tbl2"), part2.spec).spec === part2.spec) + intercept[AnalysisException] { + catalog.getPartition(TableIdentifier("tbl2"), newPart1.spec) + } + intercept[AnalysisException] { + catalog.getPartition(TableIdentifier("tbl2"), newPart2.spec) + } } test("rename partitions when database/table does not exist") { @@ -501,25 +586,31 @@ class SessionCatalogSuite extends SparkFunSuite { test("alter partitions") { val catalog = new SessionCatalog(newBasicCatalog()) - catalog.setCurrentDatabase("db2") val newLocation = newUriForDatabase() - // alter but keep spec the same - val oldPart1 = catalog.getPartition(TableIdentifier("tbl2"), part1.spec) - val oldPart2 = catalog.getPartition(TableIdentifier("tbl2"), part2.spec) - catalog.alterPartitions(TableIdentifier("tbl2"), Seq( + // Alter but keep spec the same + val oldPart1 = catalog.getPartition(TableIdentifier("tbl2", Some("db2")), part1.spec) + val oldPart2 = catalog.getPartition(TableIdentifier("tbl2", Some("db2")), part2.spec) + catalog.alterPartitions(TableIdentifier("tbl2", Some("db2")), Seq( oldPart1.copy(storage = storageFormat.copy(locationUri = Some(newLocation))), oldPart2.copy(storage = storageFormat.copy(locationUri = Some(newLocation))))) - val newPart1 = catalog.getPartition(TableIdentifier("tbl2"), part1.spec) - val newPart2 = catalog.getPartition(TableIdentifier("tbl2"), part2.spec) + val newPart1 = catalog.getPartition(TableIdentifier("tbl2", Some("db2")), part1.spec) + val newPart2 = catalog.getPartition(TableIdentifier("tbl2", Some("db2")), part2.spec) assert(newPart1.storage.locationUri == Some(newLocation)) assert(newPart2.storage.locationUri == Some(newLocation)) assert(oldPart1.storage.locationUri != Some(newLocation)) assert(oldPart2.storage.locationUri != Some(newLocation)) - // alter but change spec, should fail because new partition specs do not exist yet + // Alter partitions without explicitly specifying database + catalog.setCurrentDatabase("db2") + catalog.alterPartitions(TableIdentifier("tbl2"), Seq(oldPart1, oldPart2)) + val newerPart1 = catalog.getPartition(TableIdentifier("tbl2"), part1.spec) + val newerPart2 = catalog.getPartition(TableIdentifier("tbl2"), part2.spec) + assert(oldPart1.storage.locationUri == newerPart1.storage.locationUri) + assert(oldPart2.storage.locationUri == newerPart2.storage.locationUri) + // Alter but change spec, should fail because new partition specs do not exist yet val badPart1 = part1.copy(spec = Map("a" -> "v1", "b" -> "v2")) val badPart2 = part2.copy(spec = Map("a" -> "v3", "b" -> "v4")) intercept[AnalysisException] { - catalog.alterPartitions(TableIdentifier("tbl2"), Seq(badPart1, badPart2)) + catalog.alterPartitions(TableIdentifier("tbl2", Some("db2")), Seq(badPart1, badPart2)) } } @@ -533,6 +624,14 @@ class SessionCatalogSuite extends SparkFunSuite { } } + test("list partitions") { + val catalog = new SessionCatalog(newBasicCatalog()) + assert(catalog.listPartitions(TableIdentifier("tbl2", Some("db2"))).toSet == Set(part1, part2)) + // List partitions without explicitly specifying database + catalog.setCurrentDatabase("db2") + assert(catalog.listPartitions(TableIdentifier("tbl2")).toSet == Set(part1, part2)) + } + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- @@ -541,9 +640,12 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newEmptyCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) sessionCatalog.createDatabase(newDb("mydb"), ignoreIfExists = false) - sessionCatalog.setCurrentDatabase("mydb") - sessionCatalog.createFunction(newFunc("myfunc")) + sessionCatalog.createFunction(newFunc("myfunc", Some("mydb"))) assert(externalCatalog.listFunctions("mydb", "*").toSet == Set("myfunc")) + // Create function without explicitly specifying database + sessionCatalog.setCurrentDatabase("mydb") + sessionCatalog.createFunction(newFunc("myfunc2")) + assert(externalCatalog.listFunctions("mydb", "*").toSet == Set("myfunc", "myfunc2")) } test("create function when database does not exist") { @@ -555,7 +657,9 @@ class SessionCatalogSuite extends SparkFunSuite { test("create function that already exists") { val catalog = new SessionCatalog(newBasicCatalog()) - intercept[AnalysisException] { catalog.createFunction(newFunc("func1", Some("db2"))) } + intercept[AnalysisException] { + catalog.createFunction(newFunc("func1", Some("db2"))) + } } test("create temp function") { @@ -583,17 +687,19 @@ class SessionCatalogSuite extends SparkFunSuite { assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1")) sessionCatalog.dropFunction(FunctionIdentifier("func1", Some("db2"))) assert(externalCatalog.listFunctions("db2", "*").isEmpty) + // Drop function without explicitly specifying database + sessionCatalog.setCurrentDatabase("db2") + sessionCatalog.createFunction(newFunc("func2", Some("db2"))) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func2")) + sessionCatalog.dropFunction(FunctionIdentifier("func2")) + assert(externalCatalog.listFunctions("db2", "*").isEmpty) } - test("drop function when database does not exist") { + test("drop function when database/function does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.dropFunction(FunctionIdentifier("something", Some("does_not_exist"))) } - } - - test("drop function that does not exist") { - val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.dropFunction(FunctionIdentifier("does_not_exist")) } @@ -614,18 +720,21 @@ class SessionCatalogSuite extends SparkFunSuite { test("get function") { val catalog = new SessionCatalog(newBasicCatalog()) - assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))) == - CatalogFunction(FunctionIdentifier("func1", Some("db2")), funcClass)) - intercept[AnalysisException] { - catalog.getFunction(FunctionIdentifier("does_not_exist", Some("db2"))) - } + val expected = CatalogFunction(FunctionIdentifier("func1", Some("db2")), funcClass) + assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))) == expected) + // Get function without explicitly specifying database + catalog.setCurrentDatabase("db2") + assert(catalog.getFunction(FunctionIdentifier("func1")) == expected) } - test("get function when database does not exist") { + test("get function when database/function does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.getFunction(FunctionIdentifier("func1", Some("does_not_exist"))) } + intercept[AnalysisException] { + catalog.getFunction(FunctionIdentifier("does_not_exist", Some("db2"))) + } } test("get temp function") { @@ -648,21 +757,20 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) val newName = "funcky" - assert(sessionCatalog.getFunction(FunctionIdentifier("func1", Some("db2"))).className - == funcClass) + assert(sessionCatalog.getFunction( + FunctionIdentifier("func1", Some("db2"))) == newFunc("func1", Some("db2"))) assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1")) sessionCatalog.renameFunction( FunctionIdentifier("func1", Some("db2")), FunctionIdentifier(newName, Some("db2"))) + assert(sessionCatalog.getFunction( + FunctionIdentifier(newName, Some("db2"))) == newFunc(newName, Some("db2"))) assert(externalCatalog.listFunctions("db2", "*").toSet == Set(newName)) - assert(sessionCatalog.getFunction(FunctionIdentifier(newName, Some("db2"))).name.funcName - == newName) - assert(sessionCatalog.getFunction(FunctionIdentifier(newName, Some("db2"))).className - == funcClass) - // Renaming a non-existent function should fail - intercept[AnalysisException] { - sessionCatalog.renameFunction( - FunctionIdentifier("does_not_exist", Some("db2")), FunctionIdentifier("x", Some("db2"))) - } + // Rename function without explicitly specifying database + sessionCatalog.setCurrentDatabase("db2") + sessionCatalog.renameFunction(FunctionIdentifier(newName), FunctionIdentifier("func1")) + assert(sessionCatalog.getFunction( + FunctionIdentifier("func1")) == newFunc("func1", Some("db2"))) + assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func1")) // Renaming "db2.func1" to "db1.func2" should fail because databases don't match intercept[AnalysisException] { sessionCatalog.renameFunction( @@ -670,13 +778,18 @@ class SessionCatalogSuite extends SparkFunSuite { } } - test("rename function when database does not exist") { + test("rename function when database/function does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.renameFunction( FunctionIdentifier("func1", Some("does_not_exist")), FunctionIdentifier("func5", Some("does_not_exist"))) } + intercept[AnalysisException] { + catalog.renameFunction( + FunctionIdentifier("does_not_exist", Some("db2")), + FunctionIdentifier("x", Some("db2"))) + } } test("rename temp function") { @@ -691,8 +804,8 @@ class SessionCatalogSuite extends SparkFunSuite { assert(sessionCatalog.getTempFunction("func1") == Some(tempFunc)) assert(sessionCatalog.getTempFunction("func3") == None) assert(externalCatalog.listFunctions("db2", "*").toSet == Set("func3")) - sessionCatalog.createFunction(newFunc("func1", Some("db2"))) // If no database is specified, we'll first rename temporary functions + sessionCatalog.createFunction(newFunc("func1", Some("db2"))) sessionCatalog.renameFunction(FunctionIdentifier("func1"), FunctionIdentifier("func4")) assert(sessionCatalog.getTempFunction("func4") == Some(tempFunc.copy(name = FunctionIdentifier("func4")))) @@ -709,14 +822,20 @@ class SessionCatalogSuite extends SparkFunSuite { assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))).className == funcClass) catalog.alterFunction(newFunc("func1", Some("db2")).copy(className = "muhaha")) assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))).className == "muhaha") - intercept[AnalysisException] { catalog.alterFunction(newFunc("funcky", Some("db2"))) } + // Alter function without explicitly specifying database + catalog.setCurrentDatabase("db2") + catalog.alterFunction(newFunc("func1").copy(className = "derpy")) + assert(catalog.getFunction(FunctionIdentifier("func1")).className == "derpy") } - test("alter function when database does not exist") { + test("alter function when database/function does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[AnalysisException] { catalog.alterFunction(newFunc("func5", Some("does_not_exist"))) } + intercept[AnalysisException] { + catalog.alterFunction(newFunc("funcky", Some("db2"))) + } } test("list functions") { @@ -728,7 +847,8 @@ class SessionCatalogSuite extends SparkFunSuite { catalog.createTempFunction(tempFunc1, ignoreIfExists = false) catalog.createTempFunction(tempFunc2, ignoreIfExists = false) assert(catalog.listFunctions("db1", "*").toSet == - Set(FunctionIdentifier("func1"), FunctionIdentifier("yes_me"))) + Set(FunctionIdentifier("func1"), + FunctionIdentifier("yes_me"))) assert(catalog.listFunctions("db2", "*").toSet == Set(FunctionIdentifier("func1"), FunctionIdentifier("yes_me"),