From 3b6660578f23c69abfb59fae6796ee10bf4d482d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 13:16:30 -0800 Subject: [PATCH 01/30] Add skeleton for HiveCatalog --- .../apache/spark/sql/hive/HiveCatalog.scala | 101 ++++++++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 + 2 files changed, 103 insertions(+) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala 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 new file mode 100644 index 0000000000000..2bca1077408fe --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.catalyst.catalog.{Catalog, Database, Function, Table, TablePartition} + + +/** + * A persistent implementation of the system catalog using Hive. + * + * All public methods must be synchronized for thread-safety. + */ +private[spark] class HiveCatalog extends Catalog { + import Catalog._ + + def createDatabase(dbDefinition: Database, ignoreIfExists: Boolean): Unit = ??? + + def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = ??? + + /** + * Alter an existing database. This operation does not support renaming. + */ + def alterDatabase(db: String, dbDefinition: Database): Unit = ??? + + def getDatabase(db: String): Database = ??? + + def listDatabases(): Seq[String] = ??? + + def listDatabases(pattern: String): Seq[String] = ??? + + def createTable(db: String, tableDefinition: Table, ignoreIfExists: Boolean): Unit = ??? + + def dropTable(db: String, table: String, ignoreIfNotExists: Boolean): Unit = ??? + + def renameTable(db: String, oldName: String, newName: String): Unit = ??? + + /** + * Alter an existing table. This operation does not support renaming. + */ + def alterTable(db: String, table: String, tableDefinition: Table): Unit = ??? + + def getTable(db: String, table: String): Table = ??? + + def listTables(db: String): Seq[String] = ??? + + def listTables(db: String, pattern: String): Seq[String] = ??? + + def createPartitions( + db: String, + table: String, + parts: Seq[TablePartition], + ignoreIfExists: Boolean): Unit = ??? + + def dropPartitions( + db: String, + table: String, + parts: Seq[PartitionSpec], + ignoreIfNotExists: Boolean): Unit = ??? + + /** + * Alter an existing table partition and optionally override its spec. + */ + def alterPartition( + db: String, + table: String, + spec: PartitionSpec, + newPart: TablePartition): Unit = ??? + + def getPartition(db: String, table: String, spec: PartitionSpec): TablePartition = ??? + + def listPartitions(db: String, table: String): Seq[TablePartition] = ??? + + def createFunction(db: String, funcDefinition: Function, ignoreIfExists: Boolean): Unit = ??? + + def dropFunction(db: String, funcName: String): Unit = ??? + + /** + * Alter an existing function and optionally override its name. + */ + def alterFunction(db: String, funcName: String, funcDefinition: Function): Unit = ??? + + def getFunction(db: String, funcName: String): Function = ??? + + def listFunctions(db: String, pattern: String): Seq[String] = ??? + +} 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 61d0d6759ff72..492fff46d11c5 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 @@ -96,6 +96,8 @@ private[hive] object HiveSerDe { } } + +// TODO: replace this with o.a.s.sql.hive.HiveCatalog once we merge SQLContext and HiveContext private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveContext) extends Catalog with Logging { From f3e094ad21bd38d400f90b93898995182a508e9b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 13:34:36 -0800 Subject: [PATCH 02/30] Implement createDatabase --- .../apache/spark/sql/hive/HiveCatalog.scala | 7 ++++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/client/HiveClient.scala | 8 +++--- .../sql/hive/client/HiveClientImpl.scala | 25 +++++++++++-------- .../spark/sql/hive/MultiDatabaseSuite.scala | 2 +- .../spark/sql/hive/client/VersionsSuite.scala | 9 ++++--- 6 files changed, 31 insertions(+), 22 deletions(-) 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 2bca1077408fe..e25ca8d81d657 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.catalyst.catalog.{Catalog, Database, Function, Table, TablePartition} +import org.apache.spark.sql.hive.client.HiveClient /** @@ -25,10 +26,12 @@ import org.apache.spark.sql.catalyst.catalog.{Catalog, Database, Function, Table * * All public methods must be synchronized for thread-safety. */ -private[spark] class HiveCatalog extends Catalog { +private[spark] class HiveCatalog(client: HiveClient) extends Catalog { import Catalog._ - def createDatabase(dbDefinition: Database, ignoreIfExists: Boolean): Unit = ??? + def createDatabase(dbDefinition: Database, ignoreIfExists: Boolean): Unit = synchronized { + client.createDatabase(dbDefinition, ignoreIfExists) + } def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = ??? 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 492fff46d11c5..f7cffb0cbddfe 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 @@ -402,7 +402,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte def hiveDefaultTableFilePath(tableIdent: TableIdentifier): String = { // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent) - new Path(new Path(client.getDatabase(dbName).location), tblName).toString + new Path(new Path(client.getDatabase(dbName).locationUri), tblName).toString } override def tableExists(tableIdent: TableIdentifier): Boolean = { 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 f681cc67041a1..f15c0b45ccdec 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 @@ -22,9 +22,9 @@ import java.util.{Map => JMap} import javax.annotation.Nullable import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} +import org.apache.spark.sql.catalyst.catalog.Database import org.apache.spark.sql.catalyst.expressions.Expression -private[hive] case class HiveDatabase(name: String, location: String) private[hive] abstract class TableType { val name: String } private[hive] case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } @@ -113,12 +113,12 @@ private[hive] trait HiveClient { def setCurrentDatabase(databaseName: String): Unit /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ - def getDatabase(name: String): HiveDatabase = { + def getDatabase(name: String): Database = { getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException) } /** Returns the metadata for a given database, or None if it doesn't exist. */ - def getDatabaseOption(name: String): Option[HiveDatabase] + def getDatabaseOption(name: String): Option[Database] /** Returns the specified table, or throws [[NoSuchTableException]]. */ def getTable(dbName: String, tableName: String): HiveTable = { @@ -141,7 +141,7 @@ private[hive] trait HiveClient { def alterTable(table: HiveTable): Unit /** Creates a new database with the given name. */ - def createDatabase(database: HiveDatabase): Unit + def createDatabase(database: Database, ignoreIfExists: Boolean): Unit /** Returns the specified paritition or None if it does not exist. */ def getPartitionOption( 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 cf1ff55c96fc9..d568bea475269 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 @@ -26,7 +26,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{TableType => HTableType} -import org.apache.hadoop.hive.metastore.api.{Database, FieldSchema} +import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} import org.apache.hadoop.hive.ql.{metadata, Driver} import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.processors._ @@ -36,6 +36,7 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +import org.apache.spark.sql.catalyst.catalog.Database import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.util.{CircularBuffer, Utils} @@ -238,21 +239,25 @@ private[hive] class HiveClientImpl( } } - override def createDatabase(database: HiveDatabase): Unit = withHiveState { + override def createDatabase( + database: Database, + ignoreIfExists: Boolean): Unit = withHiveState { client.createDatabase( - new Database( + new HiveDatabase( database.name, - "", - new File(database.location).toURI.toString, - new java.util.HashMap), - true) + database.description, + database.locationUri, + database.properties.asJava), + ignoreIfExists) } - override def getDatabaseOption(name: String): Option[HiveDatabase] = withHiveState { + override def getDatabaseOption(name: String): Option[Database] = withHiveState { Option(client.getDatabase(name)).map { d => - HiveDatabase( + Database( name = d.getName, - location = d.getLocationUri) + description = d.getDescription, + locationUri = d.getLocationUri, + properties = d.getParameters.asScala.toMap) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index c2c896e5f61bb..b5f65c96aff2e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -26,7 +26,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle private def checkTablePath(dbName: String, tableName: String): Unit = { val metastoreTable = hiveContext.catalog.client.getTable(dbName, tableName) - val expectedPath = hiveContext.catalog.client.getDatabase(dbName).location + "/" + tableName + val expectedPath = hiveContext.catalog.client.getDatabase(dbName).locationUri + "/" + tableName assert(metastoreTable.serdeProperties("path") === expectedPath) } 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 1344a2cc4bd37..c707c8215f167 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, SparkFunSuite} +import org.apache.spark.sql.catalyst.catalog.Database import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal, NamedExpression} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.HiveContext @@ -60,8 +61,8 @@ class VersionsSuite extends SparkFunSuite with Logging { hadoopVersion = VersionInfo.getVersion, config = buildConf(), ivyPath = ivyPath).createClient() - val db = new HiveDatabase("default", "") - badClient.createDatabase(db) + val db = new Database("default", "desc", "loc", Map()) + badClient.createDatabase(db, ignoreIfExists = true) } private def getNestedMessages(e: Throwable): String = { @@ -116,8 +117,8 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: createDatabase") { - val db = HiveDatabase("default", "") - client.createDatabase(db) + val db = Database("default", "desc", "loc", Map()) + client.createDatabase(db, ignoreIfExists = true) } test(s"$version: createTable") { From 4b09a7da8ddcc17a813e494d868a6ea55f01cd2e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 13:48:00 -0800 Subject: [PATCH 03/30] Fix style --- .../apache/spark/sql/hive/HiveCatalog.scala | 97 ++++++++++++++----- 1 file changed, 75 insertions(+), 22 deletions(-) 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 e25ca8d81d657..b44c261ed620c 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 @@ -33,47 +33,81 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { client.createDatabase(dbDefinition, ignoreIfExists) } - def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = ??? + def dropDatabase( + db: String, + ignoreIfNotExists: Boolean, + cascade: Boolean): Unit = synchronized { + throw new UnsupportedOperationException + } /** * Alter an existing database. This operation does not support renaming. */ - def alterDatabase(db: String, dbDefinition: Database): Unit = ??? + def alterDatabase(db: String, dbDefinition: Database): Unit = synchronized { + throw new UnsupportedOperationException + } - def getDatabase(db: String): Database = ??? + def getDatabase(db: String): Database = synchronized { + throw new UnsupportedOperationException + } - def listDatabases(): Seq[String] = ??? + def listDatabases(): Seq[String] = synchronized { + throw new UnsupportedOperationException + } - def listDatabases(pattern: String): Seq[String] = ??? + def listDatabases(pattern: String): Seq[String] = synchronized { + throw new UnsupportedOperationException + } - def createTable(db: String, tableDefinition: Table, ignoreIfExists: Boolean): Unit = ??? + def createTable( + db: String, + tableDefinition: Table, + ignoreIfExists: Boolean): Unit = synchronized { + throw new UnsupportedOperationException + } - def dropTable(db: String, table: String, ignoreIfNotExists: Boolean): Unit = ??? + def dropTable(db: String, table: String, ignoreIfNotExists: Boolean): Unit = synchronized { + throw new UnsupportedOperationException + } - def renameTable(db: String, oldName: String, newName: String): Unit = ??? + def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { + throw new UnsupportedOperationException + } /** * Alter an existing table. This operation does not support renaming. */ - def alterTable(db: String, table: String, tableDefinition: Table): Unit = ??? + def alterTable(db: String, table: String, tableDefinition: Table): Unit = synchronized { + throw new UnsupportedOperationException + } - def getTable(db: String, table: String): Table = ??? + def getTable(db: String, table: String): Table = synchronized { + throw new UnsupportedOperationException + } - def listTables(db: String): Seq[String] = ??? + def listTables(db: String): Seq[String] = synchronized { + throw new UnsupportedOperationException + } - def listTables(db: String, pattern: String): Seq[String] = ??? + def listTables(db: String, pattern: String): Seq[String] = synchronized { + throw new UnsupportedOperationException + } def createPartitions( db: String, table: String, parts: Seq[TablePartition], - ignoreIfExists: Boolean): Unit = ??? + ignoreIfExists: Boolean): Unit = synchronized { + throw new UnsupportedOperationException + } def dropPartitions( db: String, table: String, parts: Seq[PartitionSpec], - ignoreIfNotExists: Boolean): Unit = ??? + ignoreIfNotExists: Boolean): Unit = synchronized { + throw new UnsupportedOperationException + } /** * Alter an existing table partition and optionally override its spec. @@ -82,23 +116,42 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, table: String, spec: PartitionSpec, - newPart: TablePartition): Unit = ??? + newPart: TablePartition): Unit = synchronized { + throw new UnsupportedOperationException + } - def getPartition(db: String, table: String, spec: PartitionSpec): TablePartition = ??? + def getPartition(db: String, table: String, spec: PartitionSpec): TablePartition = synchronized { + throw new UnsupportedOperationException + } - def listPartitions(db: String, table: String): Seq[TablePartition] = ??? + def listPartitions(db: String, table: String): Seq[TablePartition] = synchronized { + throw new UnsupportedOperationException + } - def createFunction(db: String, funcDefinition: Function, ignoreIfExists: Boolean): Unit = ??? + def createFunction( + db: String, + funcDefinition: Function, + ignoreIfExists: Boolean): Unit = synchronized { + throw new UnsupportedOperationException + } - def dropFunction(db: String, funcName: String): Unit = ??? + def dropFunction(db: String, funcName: String): Unit = synchronized { + throw new UnsupportedOperationException + } /** * Alter an existing function and optionally override its name. */ - def alterFunction(db: String, funcName: String, funcDefinition: Function): Unit = ??? + def alterFunction(db: String, funcName: String, funcDefinition: Function): Unit = synchronized { + throw new UnsupportedOperationException + } - def getFunction(db: String, funcName: String): Function = ??? + def getFunction(db: String, funcName: String): Function = synchronized { + throw new UnsupportedOperationException + } - def listFunctions(db: String, pattern: String): Seq[String] = ??? + def listFunctions(db: String, pattern: String): Seq[String] = synchronized { + throw new UnsupportedOperationException + } } From 526f278d78664c49572fd1b48495ca99d12d1896 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 13:59:02 -0800 Subject: [PATCH 04/30] Implement dropDatabase --- .../scala/org/apache/spark/sql/hive/HiveCatalog.scala | 2 +- .../org/apache/spark/sql/hive/client/HiveClient.scala | 9 +++++++++ .../apache/spark/sql/hive/client/HiveClientImpl.scala | 7 +++++++ 3 files changed, 17 insertions(+), 1 deletion(-) 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 b44c261ed620c..7cb74bc6589b8 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 @@ -37,7 +37,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + client.dropDatabase(db, ignoreIfNotExists, cascade) } /** 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 f15c0b45ccdec..3b19ab25c9516 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 @@ -143,6 +143,15 @@ private[hive] trait HiveClient { /** Creates a new database with the given name. */ def createDatabase(database: Database, ignoreIfExists: Boolean): Unit + /** + * Drop the specified database, if it exists. + * + * @param name database to drop + * @param ignoreIfNotExists if true, do not throw error if the database does not exist + * @param cascade whether to remove all associated objects such as tables and functions + */ + def dropDatabase(name: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit + /** Returns the specified paritition or None if it does not exist. */ def getPartitionOption( hTable: HiveTable, 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 d568bea475269..7ef4a34ad2366 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 @@ -251,6 +251,13 @@ private[hive] class HiveClientImpl( ignoreIfExists) } + override def dropDatabase( + name: String, + ignoreIfNotExists: Boolean, + cascade: Boolean): Unit = withHiveState { + client.dropDatabase(name, true, ignoreIfNotExists, cascade) + } + override def getDatabaseOption(name: String): Option[Database] = withHiveState { Option(client.getDatabase(name)).map { d => Database( From 4aa6e66b5ee9fa2e5f8e4b9955ed98de5b35a57c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 14:06:08 -0800 Subject: [PATCH 05/30] Implement alterDatabase --- .../apache/spark/sql/hive/HiveCatalog.scala | 18 +++++++++++++++++- .../spark/sql/hive/client/HiveClient.scala | 5 +++++ .../spark/sql/hive/client/HiveClientImpl.scala | 14 ++++++++++++++ 3 files changed, 36 insertions(+), 1 deletion(-) 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 7cb74bc6589b8..0627e1c5fc516 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 @@ -29,6 +29,10 @@ import org.apache.spark.sql.hive.client.HiveClient private[spark] class HiveCatalog(client: HiveClient) extends Catalog { import Catalog._ + // -------------------------------------------------------------------------- + // Databases + // -------------------------------------------------------------------------- + def createDatabase(dbDefinition: Database, ignoreIfExists: Boolean): Unit = synchronized { client.createDatabase(dbDefinition, ignoreIfExists) } @@ -44,7 +48,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { * Alter an existing database. This operation does not support renaming. */ def alterDatabase(db: String, dbDefinition: Database): Unit = synchronized { - throw new UnsupportedOperationException + client.alterDatabase(db, dbDefinition) } def getDatabase(db: String): Database = synchronized { @@ -59,6 +63,10 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { throw new UnsupportedOperationException } + // -------------------------------------------------------------------------- + // Tables + // -------------------------------------------------------------------------- + def createTable( db: String, tableDefinition: Table, @@ -93,6 +101,10 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { throw new UnsupportedOperationException } + // -------------------------------------------------------------------------- + // Partitions + // -------------------------------------------------------------------------- + def createPartitions( db: String, table: String, @@ -128,6 +140,10 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { throw new UnsupportedOperationException } + // -------------------------------------------------------------------------- + // Functions + // -------------------------------------------------------------------------- + def createFunction( db: String, funcDefinition: Function, 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 3b19ab25c9516..6413818e45231 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 @@ -152,6 +152,11 @@ private[hive] trait HiveClient { */ def dropDatabase(name: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit + /** + * Alter an existing database. This operation does not support renaming. + */ + def alterDatabase(name: String, database: Database): Unit + /** Returns the specified paritition or None if it does not exist. */ def getPartitionOption( hTable: HiveTable, 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 7ef4a34ad2366..97d736f31584c 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 @@ -258,6 +258,20 @@ private[hive] class HiveClientImpl( client.dropDatabase(name, true, ignoreIfNotExists, cascade) } + /** + * Alter an existing database. This operation does not support renaming. + */ + override def alterDatabase(name: String, database: Database): Unit = withHiveState { + assert(name == database.name) + client.alterDatabase( + name, + new HiveDatabase( + database.name, + database.description, + database.locationUri, + database.properties.asJava)) + } + override def getDatabaseOption(name: String): Option[Database] = withHiveState { Option(client.getDatabase(name)).map { d => Database( From 433d180260c57a905e226f0b8686eeb92d5dc938 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 14:14:15 -0800 Subject: [PATCH 06/30] Implement getDatabase, listDatabases and databaseExists --- .../spark/sql/catalyst/catalog/InMemoryCatalog.scala | 5 +++++ .../apache/spark/sql/catalyst/catalog/interface.scala | 2 ++ .../scala/org/apache/spark/sql/hive/HiveCatalog.scala | 10 +++++++--- .../org/apache/spark/sql/hive/client/HiveClient.scala | 5 ++++- .../apache/spark/sql/hive/client/HiveClientImpl.scala | 4 ++++ 5 files changed, 22 insertions(+), 4 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 38be61c52a95e..526adb3cef35c 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 @@ -39,6 +39,7 @@ class InMemoryCatalog extends Catalog { val functions = new mutable.HashMap[String, Function] } + // Database name -> spec private val catalog = new scala.collection.mutable.HashMap[String, DatabaseDesc] private def filterPattern(names: Seq[String], pattern: String): Seq[String] = { @@ -135,6 +136,10 @@ class InMemoryCatalog extends Catalog { catalog(db).db } + override def databaseExists(db: String): Boolean = synchronized { + catalog.contains(db) + } + override def listDatabases(): Seq[String] = synchronized { catalog.keySet.toSeq } 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 56aaa6bc6c2e9..22c4d7d7e59e9 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 @@ -46,6 +46,8 @@ abstract class Catalog { def getDatabase(db: String): Database + def databaseExists(db: String): Boolean + def listDatabases(): Seq[String] def listDatabases(pattern: String): Seq[String] 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 0627e1c5fc516..461818eef5d73 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 @@ -52,15 +52,19 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { } def getDatabase(db: String): Database = synchronized { - throw new UnsupportedOperationException + client.getDatabase(db) + } + + def databaseExists(db: String): Boolean = synchronized { + client.getDatabaseOption(db).isDefined } def listDatabases(): Seq[String] = synchronized { - throw new UnsupportedOperationException + client.listDatabases("*") } def listDatabases(pattern: String): Seq[String] = synchronized { - throw new UnsupportedOperationException + client.listDatabases(pattern) } // -------------------------------------------------------------------------- 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 6413818e45231..7e3d2675e270e 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 @@ -113,13 +113,16 @@ private[hive] trait HiveClient { def setCurrentDatabase(databaseName: String): Unit /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ - def getDatabase(name: String): Database = { + final def getDatabase(name: String): Database = { getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException) } /** Returns the metadata for a given database, or None if it doesn't exist. */ def getDatabaseOption(name: String): Option[Database] + /** List the names of all the databases that match the specified pattern. */ + def listDatabases(pattern: String): Seq[String] + /** Returns the specified table, or throws [[NoSuchTableException]]. */ def getTable(dbName: String, tableName: String): HiveTable = { getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException) 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 97d736f31584c..36b35e36076f1 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 @@ -282,6 +282,10 @@ private[hive] class HiveClientImpl( } } + override def listDatabases(pattern: String): Seq[String] = withHiveState { + client.getDatabasesByPattern(pattern).asScala.toSeq + } + override def getTableOption( dbName: String, tableName: String): Option[HiveTable] = withHiveState { From ff5c5bea8d4d84ae56acd4caf225e59231b946ba Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 15:18:53 -0800 Subject: [PATCH 07/30] Implement createTable This required converting o.a.s.sql.catalyst.catalog.Table to its counterpart in o.a.s.sql.hive.client.HiveTable. This required making o.a.s.sql.hive.client.TableType an enum because we need to create one of these from name. --- .../sql/catalyst/catalog/interface.scala | 29 ++++++++++--------- .../apache/spark/sql/hive/HiveCatalog.scala | 26 +++++++++++++++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 ++--- .../org/apache/spark/sql/hive/HiveQl.scala | 9 ++++-- .../spark/sql/hive/client/HiveClient.scala | 14 +++++---- .../sql/hive/client/HiveClientImpl.scala | 9 +++--- .../sql/hive/HiveMetastoreCatalogSuite.scala | 2 +- .../apache/spark/sql/hive/HiveQlSuite.scala | 3 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 3 +- .../spark/sql/hive/client/VersionsSuite.scala | 2 +- 10 files changed, 67 insertions(+), 38 deletions(-) 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 22c4d7d7e59e9..098fe043e0df9 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.types.DataType /** @@ -152,7 +153,7 @@ case class StorageFormat( */ case class Column( name: String, - dataType: String, + dataType: DataType, nullable: Boolean, comment: String ) @@ -177,19 +178,19 @@ case class TablePartition( * future once we have a better understanding of how we want to handle skewed columns. */ case class Table( - name: String, - description: String, - schema: Seq[Column], - partitionColumns: Seq[Column], - sortColumns: Seq[Column], - storage: StorageFormat, - numBuckets: Int, - properties: Map[String, String], - tableType: String, - createTime: Long, - lastAccessTime: Long, - viewOriginalText: Option[String], - viewText: Option[String]) { + name: String, + description: String, + schema: Seq[Column], + partitionColumns: Seq[Column], + sortColumns: Seq[Column], + storage: StorageFormat, + numBuckets: Int, + properties: Map[String, String], + tableType: String, + createTime: Long, + lastAccessTime: Long, + viewOriginalText: Option[String], + viewText: Option[String]) { require(tableType == "EXTERNAL_TABLE" || tableType == "INDEX_TABLE" || tableType == "MANAGED_TABLE" || tableType == "VIRTUAL_VIEW") 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 461818eef5d73..407215b730a0e 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 @@ -17,8 +17,8 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.catalyst.catalog.{Catalog, Database, Function, Table, TablePartition} -import org.apache.spark.sql.hive.client.HiveClient +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.hive.client.{HiveClient, HiveColumn, HiveTable, TableType} /** @@ -29,6 +29,26 @@ import org.apache.spark.sql.hive.client.HiveClient private[spark] class HiveCatalog(client: HiveClient) extends Catalog { import Catalog._ + private def toHiveColumn(c: Column): HiveColumn = { + HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment) + } + + private def toHiveTable(db: String, t: Table): HiveTable = { + HiveTable( + specifiedDatabase = Some(db), + name = t.name, + schema = t.schema.map(toHiveColumn), + partitionColumns = t.partitionColumns.map(toHiveColumn), + properties = t.properties, + serdeProperties = t.storage.serdeProperties, + tableType = TableType.withName(t.tableType), + location = Some(t.storage.locationUri), + inputFormat = Some(t.storage.inputFormat), + outputFormat = Some(t.storage.outputFormat), + serde = Some(t.storage.serde), + viewText = t.viewText) + } + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -75,7 +95,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, tableDefinition: Table, ignoreIfExists: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + client.createTable(toHiveTable(db, tableDefinition)) } def dropTable(db: String, table: String, ignoreIfNotExists: Boolean): Unit = synchronized { 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 f7cffb0cbddfe..6434f6bd4bdaa 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 @@ -278,10 +278,10 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val tableType = if (isExternal) { tableProperties.put("EXTERNAL", "TRUE") - ExternalTable + TableType.ExternalTable } else { tableProperties.put("EXTERNAL", "FALSE") - ManagedTable + TableType.ManagedTable } val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.hiveconf) @@ -422,7 +422,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte // Then, if alias is specified, wrap the table with a Subquery using the alias. // Otherwise, wrap the table with a Subquery using the table name. alias.map(a => Subquery(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) - } else if (table.tableType == VirtualView) { + } else if (table.tableType == TableType.VirtualView) { val viewText = table.viewText.getOrElse(sys.error("Invalid view without text.")) alias match { // because hive use things like `_c0` to build the expanded text @@ -778,7 +778,7 @@ private[hive] case class MetastoreRelation tTable.setParameters(tableParameters) table.properties.foreach { case (k, v) => tableParameters.put(k, v) } - tTable.setTableType(table.tableType.name) + tTable.setTableType(table.tableType.toString) val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tTable.setSd(sd) 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 752c037a842a8..1e6bec1527038 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 @@ -208,7 +208,7 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging partitionColumns = Seq.empty[HiveColumn], properties = properties, serdeProperties = Map[String, String](), - tableType = VirtualView, + tableType = TableType.VirtualView, location = None, inputFormat = None, outputFormat = None, @@ -376,7 +376,12 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging partitionColumns = Seq.empty[HiveColumn], properties = Map[String, String](), serdeProperties = Map[String, String](), - tableType = if (externalTable.isDefined) ExternalTable else ManagedTable, + tableType = + if (externalTable.isDefined) { + TableType.ExternalTable + } else { + TableType.ManagedTable + }, location = None, inputFormat = None, outputFormat = None, 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 7e3d2675e270e..cb60f00b7d27b 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 @@ -26,11 +26,13 @@ import org.apache.spark.sql.catalyst.catalog.Database import org.apache.spark.sql.catalyst.expressions.Expression -private[hive] abstract class TableType { val name: String } -private[hive] case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } -private[hive] case object IndexTable extends TableType { override val name = "INDEX_TABLE" } -private[hive] case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } -private[hive] case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } +private[hive] object TableType extends Enumeration { + type TableType = Value + val ExternalTable = Value("EXTERNAL_TABLE") + val IndexTable = Value("INDEX_TABLE") + val ManagedTable = Value("MANAGED_TABLE") + val VirtualView = Value("VIRTUAL_VIEW") +} // TODO: Use this for Tables and Partitions private[hive] case class HiveStorageDescriptor( @@ -52,7 +54,7 @@ private[hive] case class HiveTable( partitionColumns: Seq[HiveColumn], properties: Map[String, String], serdeProperties: Map[String, String], - tableType: TableType, + tableType: TableType.Value, location: Option[String] = None, inputFormat: Option[String] = None, outputFormat: Option[String] = None, 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 36b35e36076f1..c183c35542ca9 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 @@ -294,7 +294,6 @@ private[hive] class HiveClientImpl( val hiveTable = Option(client.getTable(dbName, tableName, false)) val converted = hiveTable.map { h => - HiveTable( name = h.getTableName, specifiedDatabase = Option(h.getDbName), @@ -304,10 +303,10 @@ private[hive] class HiveClientImpl( properties = h.getParameters.asScala.toMap, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap, tableType = h.getTableType match { - case HTableType.MANAGED_TABLE => ManagedTable - case HTableType.EXTERNAL_TABLE => ExternalTable - case HTableType.VIRTUAL_VIEW => VirtualView - case HTableType.INDEX_TABLE => IndexTable + case HTableType.MANAGED_TABLE => TableType.ManagedTable + case HTableType.EXTERNAL_TABLE => TableType.ExternalTable + case HTableType.VIRTUAL_VIEW => TableType.VirtualView + case HTableType.INDEX_TABLE => TableType.IndexTable }, location = shim.getDataLocation(h), inputFormat = Option(h.getInputFormatClass).map(_.getName), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 14a83d53904a6..66502bf011a46 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -21,7 +21,7 @@ import java.io.File import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{QueryTest, Row, SaveMode, SQLConf} -import org.apache.spark.sql.hive.client.{ExternalTable, ManagedTable} +import org.apache.spark.sql.hive.client.TableType.{ExternalTable, ManagedTable} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} import org.apache.spark.sql.types.{DecimalType, StringType, StructType} 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 137dadd6c6bb3..2d1db6ee115bf 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 @@ -25,7 +25,8 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.SimpleParserConf import org.apache.spark.sql.catalyst.plans.logical.Generate -import org.apache.spark.sql.hive.client.{ExternalTable, HiveColumn, HiveTable, ManagedTable} +import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} +import org.apache.spark.sql.hive.client.TableType.{ExternalTable, ManagedTable} class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { val parser = new HiveQl(SimpleParserConf()) 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 d9e4b020fdfcc..546b3ed5b87d5 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 @@ -27,7 +27,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation -import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable} +import org.apache.spark.sql.hive.client.HiveTable +import org.apache.spark.sql.hive.client.TableType.ManagedTable import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ 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 c707c8215f167..5392e6d565052 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 @@ -130,7 +130,7 @@ class VersionsSuite extends SparkFunSuite with Logging { partitionColumns = Seq.empty, properties = Map.empty, serdeProperties = Map.empty, - tableType = ManagedTable, + tableType = TableType.ManagedTable, location = None, inputFormat = Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), From ff49f0cf6fabc645121b43b5746017c838a3551d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 15:22:38 -0800 Subject: [PATCH 08/30] Explicitly mark methods with override in HiveCatalog --- .../apache/spark/sql/hive/HiveCatalog.scala | 62 ++++++++++++------- 1 file changed, 38 insertions(+), 24 deletions(-) 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 407215b730a0e..8b608e897f762 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 @@ -53,11 +53,13 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { // Databases // -------------------------------------------------------------------------- - def createDatabase(dbDefinition: Database, ignoreIfExists: Boolean): Unit = synchronized { + override def createDatabase( + dbDefinition: Database, + ignoreIfExists: Boolean): Unit = synchronized { client.createDatabase(dbDefinition, ignoreIfExists) } - def dropDatabase( + override def dropDatabase( db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = synchronized { @@ -67,23 +69,23 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { /** * Alter an existing database. This operation does not support renaming. */ - def alterDatabase(db: String, dbDefinition: Database): Unit = synchronized { + override def alterDatabase(db: String, dbDefinition: Database): Unit = synchronized { client.alterDatabase(db, dbDefinition) } - def getDatabase(db: String): Database = synchronized { + override def getDatabase(db: String): Database = synchronized { client.getDatabase(db) } - def databaseExists(db: String): Boolean = synchronized { + override def databaseExists(db: String): Boolean = synchronized { client.getDatabaseOption(db).isDefined } - def listDatabases(): Seq[String] = synchronized { + override def listDatabases(): Seq[String] = synchronized { client.listDatabases("*") } - def listDatabases(pattern: String): Seq[String] = synchronized { + override def listDatabases(pattern: String): Seq[String] = synchronized { client.listDatabases(pattern) } @@ -91,37 +93,43 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { // Tables // -------------------------------------------------------------------------- - def createTable( + override def createTable( db: String, tableDefinition: Table, ignoreIfExists: Boolean): Unit = synchronized { client.createTable(toHiveTable(db, tableDefinition)) } - def dropTable(db: String, table: String, ignoreIfNotExists: Boolean): Unit = synchronized { + override def dropTable( + db: String, + table: String, + ignoreIfNotExists: Boolean): Unit = synchronized { throw new UnsupportedOperationException } - def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { + override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { throw new UnsupportedOperationException } /** * Alter an existing table. This operation does not support renaming. */ - def alterTable(db: String, table: String, tableDefinition: Table): Unit = synchronized { + override def alterTable( + db: String, + table: String, + tableDefinition: Table): Unit = synchronized { throw new UnsupportedOperationException } - def getTable(db: String, table: String): Table = synchronized { + override def getTable(db: String, table: String): Table = synchronized { throw new UnsupportedOperationException } - def listTables(db: String): Seq[String] = synchronized { + override def listTables(db: String): Seq[String] = synchronized { throw new UnsupportedOperationException } - def listTables(db: String, pattern: String): Seq[String] = synchronized { + override def listTables(db: String, pattern: String): Seq[String] = synchronized { throw new UnsupportedOperationException } @@ -129,7 +137,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { // Partitions // -------------------------------------------------------------------------- - def createPartitions( + override def createPartitions( db: String, table: String, parts: Seq[TablePartition], @@ -137,7 +145,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { throw new UnsupportedOperationException } - def dropPartitions( + override def dropPartitions( db: String, table: String, parts: Seq[PartitionSpec], @@ -148,7 +156,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { /** * Alter an existing table partition and optionally override its spec. */ - def alterPartition( + override def alterPartition( db: String, table: String, spec: PartitionSpec, @@ -156,11 +164,14 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { throw new UnsupportedOperationException } - def getPartition(db: String, table: String, spec: PartitionSpec): TablePartition = synchronized { + override def getPartition( + db: String, + table: String, + spec: PartitionSpec): TablePartition = synchronized { throw new UnsupportedOperationException } - def listPartitions(db: String, table: String): Seq[TablePartition] = synchronized { + override def listPartitions(db: String, table: String): Seq[TablePartition] = synchronized { throw new UnsupportedOperationException } @@ -168,29 +179,32 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { // Functions // -------------------------------------------------------------------------- - def createFunction( + override def createFunction( db: String, funcDefinition: Function, ignoreIfExists: Boolean): Unit = synchronized { throw new UnsupportedOperationException } - def dropFunction(db: String, funcName: String): Unit = synchronized { + override def dropFunction(db: String, funcName: String): Unit = synchronized { throw new UnsupportedOperationException } /** * Alter an existing function and optionally override its name. */ - def alterFunction(db: String, funcName: String, funcDefinition: Function): Unit = synchronized { + override def alterFunction( + db: String, + funcName: String, + funcDefinition: Function): Unit = synchronized { throw new UnsupportedOperationException } - def getFunction(db: String, funcName: String): Function = synchronized { + override def getFunction(db: String, funcName: String): Function = synchronized { throw new UnsupportedOperationException } - def listFunctions(db: String, pattern: String): Seq[String] = synchronized { + override def listFunctions(db: String, pattern: String): Seq[String] = synchronized { throw new UnsupportedOperationException } From ca98c00264564717ddd427282bfff301ebdb6c70 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 15:25:27 -0800 Subject: [PATCH 09/30] Implement dropTable --- .../main/scala/org/apache/spark/sql/hive/HiveCatalog.scala | 2 +- .../org/apache/spark/sql/hive/client/HiveClient.scala | 3 +++ .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 7 +++++++ 3 files changed, 11 insertions(+), 1 deletion(-) 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 8b608e897f762..4f1a88f0e000b 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 @@ -104,7 +104,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, table: String, ignoreIfNotExists: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + client.dropTable(db, table, ignoreIfNotExists) } override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { 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 cb60f00b7d27b..1b6f05fd85cb9 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 @@ -142,6 +142,9 @@ private[hive] trait HiveClient { /** Creates a table with the given metadata. */ def createTable(table: HiveTable): Unit + /** Drop the specified table. */ + def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean): Unit + /** Updates the given table with new metadata. */ def alterTable(table: HiveTable): 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 c183c35542ca9..35f16bb99f93a 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 @@ -382,6 +382,13 @@ private[hive] class HiveClientImpl( client.createTable(qlTable) } + override def dropTable( + dbName: String, + tableName: String, + ignoreIfNotExists: Boolean): Unit = withHiveState { + client.dropTable(dbName, tableName, true, ignoreIfNotExists) + } + override def alterTable(table: HiveTable): Unit = withHiveState { val qlTable = toQlTable(table) client.alterTable(table.qualifiedName, qlTable) From 71f99646cdf30a68a8e592b80ef5a6f40685551b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Feb 2016 15:40:37 -0800 Subject: [PATCH 10/30] Implement renameTable, alterTable --- .../main/scala/org/apache/spark/sql/hive/HiveCatalog.scala | 5 +++-- .../org/apache/spark/sql/hive/client/HiveClient.scala | 7 ++++++- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 5 +++-- 3 files changed, 12 insertions(+), 5 deletions(-) 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 4f1a88f0e000b..a1a02a56dab67 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 @@ -108,7 +108,8 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { } override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { - throw new UnsupportedOperationException + val hiveTable = toHiveTable(db, getTable(db, oldName)).copy(name = newName) + client.alterTable(oldName, hiveTable) } /** @@ -118,7 +119,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, table: String, tableDefinition: Table): Unit = synchronized { - throw new UnsupportedOperationException + client.alterTable(table, toHiveTable(db, tableDefinition)) } override def getTable(db: String, table: String): Table = synchronized { 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 1b6f05fd85cb9..e061d5b3203e9 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 @@ -146,7 +146,12 @@ private[hive] trait HiveClient { def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean): Unit /** Updates the given table with new metadata. */ - def alterTable(table: HiveTable): Unit + final def alterTable(table: HiveTable): Unit = { + alterTable(table.qualifiedName, table) + } + + /** Updates the given table with new metadata, optionally renaming the table. */ + def alterTable(tableName: String, table: HiveTable): Unit /** Creates a new database with the given name. */ def createDatabase(database: Database, ignoreIfExists: Boolean): 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 35f16bb99f93a..cd76da3df2150 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 @@ -389,9 +389,10 @@ private[hive] class HiveClientImpl( client.dropTable(dbName, tableName, true, ignoreIfNotExists) } - override def alterTable(table: HiveTable): Unit = withHiveState { + override def alterTable(tableName: String, table: HiveTable): Unit = withHiveState { val qlTable = toQlTable(table) - client.alterTable(table.qualifiedName, qlTable) + val qualifiedTableName = s"${table.database}.$tableName" + client.alterTable(qualifiedTableName, qlTable) } private def toHivePartition(partition: metadata.Partition): HivePartition = { From 13795d83c325a69fb35260c300b379e2e55725aa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 11 Feb 2016 16:51:36 -0800 Subject: [PATCH 11/30] Remove intermediate representation of tables, columns etc. Currently there's the catalog table, the Spark table used in the hive module, and the Hive table. To avoid converting to and from between these table representations, we kill the intermediate one, which is the one currently used throughout HiveClient and friends. --- .../catalyst/catalog/InMemoryCatalog.scala | 47 +++-- .../sql/catalyst/catalog/interface.scala | 110 +++++------ .../catalyst/catalog/CatalogTestCases.scala | 40 ++-- .../apache/spark/sql/hive/HiveCatalog.scala | 63 +++--- .../spark/sql/hive/HiveMetastoreCatalog.scala | 147 +++++++------- .../org/apache/spark/sql/hive/HiveQl.scala | 171 +++++++++------- .../spark/sql/hive/client/HiveClient.scala | 83 ++------ .../sql/hive/client/HiveClientImpl.scala | 184 +++++++++--------- .../hive/execution/CreateTableAsSelect.scala | 22 +-- .../hive/execution/CreateViewAsSelect.scala | 14 +- .../sql/hive/HiveMetastoreCatalogSuite.scala | 41 ++-- .../apache/spark/sql/hive/HiveQlSuite.scala | 95 ++++----- .../sql/hive/MetastoreDataSourcesSuite.scala | 23 ++- .../spark/sql/hive/MultiDatabaseSuite.scala | 2 +- .../spark/sql/hive/client/VersionsSuite.scala | 30 ++- .../sql/hive/execution/PruningSuite.scala | 2 +- 16 files changed, 530 insertions(+), 544 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 526adb3cef35c..7fc076fd41336 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 @@ -30,13 +30,13 @@ import org.apache.spark.sql.AnalysisException class InMemoryCatalog extends Catalog { import Catalog._ - private class TableDesc(var table: Table) { - val partitions = new mutable.HashMap[PartitionSpec, TablePartition] + private class TableDesc(var table: CatalogTable) { + val partitions = new mutable.HashMap[TablePartitionSpec, CatalogTablePartition] } - private class DatabaseDesc(var db: Database) { + private class DatabaseDesc(var db: CatalogDatabase) { val tables = new mutable.HashMap[String, TableDesc] - val functions = new mutable.HashMap[String, Function] + val functions = new mutable.HashMap[String, CatalogFunction] } // Database name -> spec @@ -57,7 +57,7 @@ class InMemoryCatalog extends Catalog { catalog(db).tables.contains(table) } - private def existsPartition(db: String, table: String, spec: PartitionSpec): Boolean = { + private def existsPartition(db: String, table: String, spec: TablePartitionSpec): Boolean = { assertTableExists(db, table) catalog(db).tables(table).partitions.contains(spec) } @@ -80,7 +80,7 @@ class InMemoryCatalog extends Catalog { } } - private def assertPartitionExists(db: String, table: String, spec: PartitionSpec): Unit = { + private def assertPartitionExists(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") } @@ -91,7 +91,7 @@ class InMemoryCatalog extends Catalog { // -------------------------------------------------------------------------- override def createDatabase( - dbDefinition: Database, + dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = synchronized { if (catalog.contains(dbDefinition.name)) { if (!ignoreIfExists) { @@ -125,13 +125,13 @@ class InMemoryCatalog extends Catalog { } } - override def alterDatabase(db: String, dbDefinition: Database): Unit = synchronized { + override def alterDatabase(db: String, dbDefinition: CatalogDatabase): Unit = synchronized { assertDbExists(db) assert(db == dbDefinition.name) catalog(db).db = dbDefinition } - override def getDatabase(db: String): Database = synchronized { + override def getDatabase(db: String): CatalogDatabase = synchronized { assertDbExists(db) catalog(db).db } @@ -154,7 +154,7 @@ class InMemoryCatalog extends Catalog { override def createTable( db: String, - tableDefinition: Table, + tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = synchronized { assertDbExists(db) if (existsTable(db, tableDefinition.name)) { @@ -188,13 +188,16 @@ class InMemoryCatalog extends Catalog { catalog(db).tables.remove(oldName) } - override def alterTable(db: String, table: String, tableDefinition: Table): Unit = synchronized { + override def alterTable( + db: String, + table: String, + tableDefinition: CatalogTable): Unit = synchronized { assertTableExists(db, table) assert(table == tableDefinition.name) catalog(db).tables(table).table = tableDefinition } - override def getTable(db: String, table: String): Table = synchronized { + override def getTable(db: String, table: String): CatalogTable = synchronized { assertTableExists(db, table) catalog(db).tables(table).table } @@ -216,7 +219,7 @@ class InMemoryCatalog extends Catalog { override def createPartitions( db: String, table: String, - parts: Seq[TablePartition], + parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = synchronized { assertTableExists(db, table) val existingParts = catalog(db).tables(table).partitions @@ -234,7 +237,7 @@ class InMemoryCatalog extends Catalog { override def dropPartitions( db: String, table: String, - partSpecs: Seq[PartitionSpec], + partSpecs: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = synchronized { assertTableExists(db, table) val existingParts = catalog(db).tables(table).partitions @@ -252,8 +255,8 @@ class InMemoryCatalog extends Catalog { override def alterPartition( db: String, table: String, - spec: Map[String, String], - newPart: TablePartition): Unit = synchronized { + spec: TablePartitionSpec, + newPart: CatalogTablePartition): Unit = synchronized { assertPartitionExists(db, table, spec) val existingParts = catalog(db).tables(table).partitions if (spec != newPart.spec) { @@ -266,12 +269,14 @@ class InMemoryCatalog extends Catalog { override def getPartition( db: String, table: String, - spec: Map[String, String]): TablePartition = synchronized { + spec: TablePartitionSpec): CatalogTablePartition = synchronized { assertPartitionExists(db, table, spec) catalog(db).tables(table).partitions(spec) } - override def listPartitions(db: String, table: String): Seq[TablePartition] = synchronized { + override def listPartitions( + db: String, + table: String): Seq[CatalogTablePartition] = synchronized { assertTableExists(db, table) catalog(db).tables(table).partitions.values.toSeq } @@ -282,7 +287,7 @@ class InMemoryCatalog extends Catalog { override def createFunction( db: String, - func: Function, + func: CatalogFunction, ignoreIfExists: Boolean): Unit = synchronized { assertDbExists(db) if (existsFunction(db, func.name)) { @@ -302,7 +307,7 @@ class InMemoryCatalog extends Catalog { override def alterFunction( db: String, funcName: String, - funcDefinition: Function): Unit = synchronized { + funcDefinition: CatalogFunction): Unit = synchronized { assertFunctionExists(db, funcName) if (funcName != funcDefinition.name) { // Also a rename; remove the old one and add the new one back @@ -311,7 +316,7 @@ class InMemoryCatalog extends Catalog { catalog(db).functions.put(funcDefinition.name, funcDefinition) } - override def getFunction(db: String, funcName: String): Function = synchronized { + override def getFunction(db: String, funcName: String): CatalogFunction = synchronized { assertFunctionExists(db, funcName) catalog(db).functions(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 098fe043e0df9..f640d8980f036 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 @@ -36,16 +36,16 @@ abstract class Catalog { // Databases // -------------------------------------------------------------------------- - def createDatabase(dbDefinition: Database, ignoreIfExists: Boolean): Unit + def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit /** * Alter an existing database. This operation does not support renaming. */ - def alterDatabase(db: String, dbDefinition: Database): Unit + def alterDatabase(db: String, dbDefinition: CatalogDatabase): Unit - def getDatabase(db: String): Database + def getDatabase(db: String): CatalogDatabase def databaseExists(db: String): Boolean @@ -57,7 +57,7 @@ abstract class Catalog { // Tables // -------------------------------------------------------------------------- - def createTable(db: String, tableDefinition: Table, ignoreIfExists: Boolean): Unit + def createTable(db: String, tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit def dropTable(db: String, table: String, ignoreIfNotExists: Boolean): Unit @@ -66,9 +66,9 @@ abstract class Catalog { /** * Alter an existing table. This operation does not support renaming. */ - def alterTable(db: String, table: String, tableDefinition: Table): Unit + def alterTable(db: String, table: String, tableDefinition: CatalogTable): Unit - def getTable(db: String, table: String): Table + def getTable(db: String, table: String): CatalogTable def listTables(db: String): Seq[String] @@ -81,13 +81,13 @@ abstract class Catalog { def createPartitions( db: String, table: String, - parts: Seq[TablePartition], + parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit def dropPartitions( db: String, table: String, - parts: Seq[PartitionSpec], + parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit /** @@ -96,28 +96,28 @@ abstract class Catalog { def alterPartition( db: String, table: String, - spec: PartitionSpec, - newPart: TablePartition): Unit + spec: TablePartitionSpec, + newPart: CatalogTablePartition): Unit - def getPartition(db: String, table: String, spec: PartitionSpec): TablePartition + def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition // TODO: support listing by pattern - def listPartitions(db: String, table: String): Seq[TablePartition] + def listPartitions(db: String, table: String): Seq[CatalogTablePartition] // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- - def createFunction(db: String, funcDefinition: Function, ignoreIfExists: Boolean): Unit + def createFunction(db: String, funcDefinition: CatalogFunction, ignoreIfExists: Boolean): Unit def dropFunction(db: String, funcName: String): Unit /** * Alter an existing function and optionally override its name. */ - def alterFunction(db: String, funcName: String, funcDefinition: Function): Unit + def alterFunction(db: String, funcName: String, funcDefinition: CatalogFunction): Unit - def getFunction(db: String, funcName: String): Function + def getFunction(db: String, funcName: String): CatalogFunction def listFunctions(db: String, pattern: String): Seq[String] @@ -130,33 +130,28 @@ abstract class Catalog { * @param name name of the function * @param className fully qualified class name, e.g. "org.apache.spark.util.MyFunc" */ -case class Function( - name: String, - className: String -) +case class CatalogFunction(name: String, className: String) /** * Storage format, used to describe how a partition or a table is stored. */ -case class StorageFormat( - locationUri: String, - inputFormat: String, - outputFormat: String, - serde: String, - serdeProperties: Map[String, String] -) +case class CatalogStorageFormat( + locationUri: Option[String], + inputFormat: Option[String], + outputFormat: Option[String], + serde: Option[String], + serdeProperties: Map[String, String]) /** * A column in a table. */ -case class Column( - name: String, - dataType: DataType, - nullable: Boolean, - comment: String -) +case class CatalogColumn( + name: String, + dataType: DataType, + nullable: Boolean = true, + comment: Option[String] = None) /** @@ -165,10 +160,7 @@ case class Column( * @param spec partition spec values indexed by column name * @param storage storage format of the partition */ -case class TablePartition( - spec: Catalog.PartitionSpec, - storage: StorageFormat -) +case class CatalogTablePartition(spec: Catalog.TablePartitionSpec, storage: CatalogStorageFormat) /** @@ -177,40 +169,42 @@ case class TablePartition( * Note that Hive's metastore also tracks skewed columns. We should consider adding that in the * future once we have a better understanding of how we want to handle skewed columns. */ -case class Table( +case class CatalogTable( + specifiedDatabase: Option[String], name: String, - description: String, - schema: Seq[Column], - partitionColumns: Seq[Column], - sortColumns: Seq[Column], - storage: StorageFormat, - numBuckets: Int, - properties: Map[String, String], tableType: String, - createTime: Long, - lastAccessTime: Long, - viewOriginalText: Option[String], - viewText: Option[String]) { + storage: CatalogStorageFormat, + schema: Seq[CatalogColumn], + partitionColumns: Seq[CatalogColumn] = Seq.empty, + sortColumns: Seq[CatalogColumn] = Seq.empty, + numBuckets: Int = 0, + createTime: Long = System.currentTimeMillis, + lastAccessTime: Long = System.currentTimeMillis, + properties: Map[String, String] = Map.empty, + viewOriginalText: Option[String] = None, + viewText: Option[String] = None) { require(tableType == "EXTERNAL_TABLE" || tableType == "INDEX_TABLE" || tableType == "MANAGED_TABLE" || tableType == "VIRTUAL_VIEW") + + /** Return the database this table was specified to belong to, assuming it exists. */ + def database: String = specifiedDatabase.getOrElse(sys.error("database not resolved")) + + /** Return the fully qualified name of this table, assuming the database was specified. */ + def qualifiedName: String = s"$database.$name" } /** * A database defined in the catalog. */ -case class Database( - name: String, - description: String, - locationUri: String, - properties: Map[String, String] -) +case class CatalogDatabase( + name: String, + description: String, + locationUri: String, + properties: Map[String, String]) object Catalog { - /** - * Specifications of a table partition indexed by column name. - */ - type PartitionSpec = Map[String, String] + type TablePartitionSpec = Seq[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 45c5ceecb0eef..a217b120c9084 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 @@ -27,10 +27,10 @@ import org.apache.spark.sql.AnalysisException * Implementations of the [[Catalog]] interface can create test suites by extending this. */ abstract class CatalogTestCases extends SparkFunSuite { - private val storageFormat = StorageFormat("usa", "$", "zzz", "serde", Map()) - private val part1 = TablePartition(Map("a" -> "1"), storageFormat) - private val part2 = TablePartition(Map("b" -> "2"), storageFormat) - private val part3 = TablePartition(Map("c" -> "3"), storageFormat) + private val storageFormat = CatalogStorageFormat(None, Some("z"), Some("y"), Some("x"), Map()) + private val part1 = CatalogTablePartition(Seq("1"), storageFormat) + private val part2 = CatalogTablePartition(Seq("2"), storageFormat) + private val part3 = CatalogTablePartition(Seq("3"), storageFormat) private val funcClass = "org.apache.spark.myFunc" protected def newEmptyCatalog(): Catalog @@ -57,16 +57,26 @@ abstract class CatalogTestCases extends SparkFunSuite { catalog } - private def newFunc(): Function = Function("funcname", funcClass) + private def newFunc(): CatalogFunction = CatalogFunction("funcname", funcClass) - private def newDb(name: String = "default"): Database = - Database(name, name + " description", "uri", Map.empty) + private def newDb(name: String = "default"): CatalogDatabase = + CatalogDatabase(name, name + " description", "uri", Map.empty) - private def newTable(name: String): Table = - Table(name, "", Seq.empty, Seq.empty, Seq.empty, null, 0, Map.empty, "EXTERNAL_TABLE", 0, 0, - None, None) + private def newTable(name: String): CatalogTable = + CatalogTable( + specifiedDatabase = Some("db"), + name = name, + tableType = "EXTERNAL_TABLE", + storage = CatalogStorageFormat( + locationUri = Some("just works"), + inputFormat = Some("this part will change"), + outputFormat = Some("just one line"), + serde = Some("if I stopped the stream"), + serdeProperties = Map.empty + ), + schema = Seq.empty) - private def newFunc(name: String): Function = Function(name, funcClass) + private def newFunc(name: String): CatalogFunction = CatalogFunction(name, funcClass) // -------------------------------------------------------------------------- // Databases @@ -144,13 +154,13 @@ abstract class CatalogTestCases extends SparkFunSuite { test("alter database") { val catalog = newBasicCatalog() - catalog.alterDatabase("db1", Database("db1", "new description", "lll", Map.empty)) + catalog.alterDatabase("db1", CatalogDatabase("db1", "new description", "lll", Map.empty)) assert(catalog.getDatabase("db1").description == "new description") } test("alter database should throw exception when the database does not exist") { intercept[AnalysisException] { - newBasicCatalog().alterDatabase("no_db", Database("no_db", "ddd", "lll", Map.empty)) + newBasicCatalog().alterDatabase("no_db", CatalogDatabase("no_db", "ddd", "lll", Map.empty)) } } @@ -336,8 +346,8 @@ abstract class CatalogTestCases extends SparkFunSuite { test("alter partitions") { val catalog = newBasicCatalog() - val partSameSpec = part1.copy(storage = storageFormat.copy(serde = "myserde")) - val partNewSpec = part1.copy(spec = Map("x" -> "10")) + val partSameSpec = part1.copy(storage = storageFormat.copy(serde = Some("myserde"))) + val partNewSpec = part1.copy(spec = Seq("10")) // alter but keep spec the same catalog.alterPartition("db2", "tbl2", part1.spec, partSameSpec) assert(catalog.getPartition("db2", "tbl2", part1.spec) == partSameSpec) 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 a1a02a56dab67..27a4048e370b0 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.hive.client.{HiveClient, HiveColumn, HiveTable, TableType} +import org.apache.spark.sql.hive.client.HiveClient /** @@ -29,32 +29,12 @@ import org.apache.spark.sql.hive.client.{HiveClient, HiveColumn, HiveTable, Tabl private[spark] class HiveCatalog(client: HiveClient) extends Catalog { import Catalog._ - private def toHiveColumn(c: Column): HiveColumn = { - HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment) - } - - private def toHiveTable(db: String, t: Table): HiveTable = { - HiveTable( - specifiedDatabase = Some(db), - name = t.name, - schema = t.schema.map(toHiveColumn), - partitionColumns = t.partitionColumns.map(toHiveColumn), - properties = t.properties, - serdeProperties = t.storage.serdeProperties, - tableType = TableType.withName(t.tableType), - location = Some(t.storage.locationUri), - inputFormat = Some(t.storage.inputFormat), - outputFormat = Some(t.storage.outputFormat), - serde = Some(t.storage.serde), - viewText = t.viewText) - } - // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- override def createDatabase( - dbDefinition: Database, + dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = synchronized { client.createDatabase(dbDefinition, ignoreIfExists) } @@ -69,11 +49,11 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { /** * Alter an existing database. This operation does not support renaming. */ - override def alterDatabase(db: String, dbDefinition: Database): Unit = synchronized { + override def alterDatabase(db: String, dbDefinition: CatalogDatabase): Unit = synchronized { client.alterDatabase(db, dbDefinition) } - override def getDatabase(db: String): Database = synchronized { + override def getDatabase(db: String): CatalogDatabase = synchronized { client.getDatabase(db) } @@ -95,21 +75,20 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def createTable( db: String, - tableDefinition: Table, + tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = synchronized { - client.createTable(toHiveTable(db, tableDefinition)) + throw new UnsupportedOperationException } override def dropTable( db: String, table: String, ignoreIfNotExists: Boolean): Unit = synchronized { - client.dropTable(db, table, ignoreIfNotExists) + throw new UnsupportedOperationException } override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { - val hiveTable = toHiveTable(db, getTable(db, oldName)).copy(name = newName) - client.alterTable(oldName, hiveTable) + throw new UnsupportedOperationException } /** @@ -118,11 +97,11 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def alterTable( db: String, table: String, - tableDefinition: Table): Unit = synchronized { - client.alterTable(table, toHiveTable(db, tableDefinition)) + tableDefinition: CatalogTable): Unit = synchronized { + throw new UnsupportedOperationException } - override def getTable(db: String, table: String): Table = synchronized { + override def getTable(db: String, table: String): CatalogTable = synchronized { throw new UnsupportedOperationException } @@ -141,7 +120,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def createPartitions( db: String, table: String, - parts: Seq[TablePartition], + parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = synchronized { throw new UnsupportedOperationException } @@ -149,7 +128,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def dropPartitions( db: String, table: String, - parts: Seq[PartitionSpec], + parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = synchronized { throw new UnsupportedOperationException } @@ -160,19 +139,21 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def alterPartition( db: String, table: String, - spec: PartitionSpec, - newPart: TablePartition): Unit = synchronized { + spec: TablePartitionSpec, + newPart: CatalogTablePartition): Unit = synchronized { throw new UnsupportedOperationException } override def getPartition( db: String, table: String, - spec: PartitionSpec): TablePartition = synchronized { + spec: TablePartitionSpec): CatalogTablePartition = synchronized { throw new UnsupportedOperationException } - override def listPartitions(db: String, table: String): Seq[TablePartition] = synchronized { + override def listPartitions( + db: String, + table: String): Seq[CatalogTablePartition] = synchronized { throw new UnsupportedOperationException } @@ -182,7 +163,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def createFunction( db: String, - funcDefinition: Function, + funcDefinition: CatalogFunction, ignoreIfExists: Boolean): Unit = synchronized { throw new UnsupportedOperationException } @@ -197,11 +178,11 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def alterFunction( db: String, funcName: String, - funcDefinition: Function): Unit = synchronized { + funcDefinition: CatalogFunction): Unit = synchronized { throw new UnsupportedOperationException } - override def getFunction(db: String, funcName: String): Function = synchronized { + override def getFunction(db: String, funcName: String): CatalogFunction = synchronized { throw new UnsupportedOperationException } 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 6434f6bd4bdaa..b69a3dda3de3d 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 @@ -27,13 +27,14 @@ import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.Warehouse import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.ql.metadata._ +import org.apache.hadoop.hive.ql.metadata.{Table => HiveTable, _} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.Logging import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ @@ -109,16 +110,16 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte /** A fully qualified identifier for a table (i.e., database.tableName) */ case class QualifiedTableName(database: String, name: String) - private def getQualifiedTableName(tableIdent: TableIdentifier) = { + private def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = { QualifiedTableName( tableIdent.database.getOrElse(client.currentDatabase).toLowerCase, tableIdent.table.toLowerCase) } - private def getQualifiedTableName(hiveTable: HiveTable) = { + private def getQualifiedTableName(t: CatalogTable): QualifiedTableName = { QualifiedTableName( - hiveTable.specifiedDatabase.getOrElse(client.currentDatabase).toLowerCase, - hiveTable.name.toLowerCase) + t.specifiedDatabase.getOrElse(client.currentDatabase).toLowerCase, + t.name.toLowerCase) } /** A cache of Spark SQL data source tables that have been accessed. */ @@ -177,7 +178,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte // It does not appear that the ql client for the metastore has a way to enumerate all the // SerDe properties directly... - val options = table.serdeProperties + val options = table.storage.serdeProperties val resolvedRelation = ResolvedDataSource( @@ -288,43 +289,42 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val dataSource = ResolvedDataSource( hive, userSpecifiedSchema, partitionColumns, bucketSpec, provider, options) - def newSparkSQLSpecificMetastoreTable(): HiveTable = { - HiveTable( + def newSparkSQLSpecificMetastoreTable(): CatalogTable = { + CatalogTable( specifiedDatabase = Option(dbName), name = tblName, + tableType = tableType.toString, schema = Nil, - partitionColumns = Nil, - tableType = tableType, - properties = tableProperties.toMap, - serdeProperties = options) + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + serdeProperties = options + ), + properties = tableProperties.toMap) } - def newHiveCompatibleMetastoreTable(relation: HadoopFsRelation, serde: HiveSerDe): HiveTable = { - def schemaToHiveColumn(schema: StructType): Seq[HiveColumn] = { - schema.map { field => - HiveColumn( - name = field.name, - hiveType = HiveMetastoreTypes.toMetastoreType(field.dataType), - comment = "") - } - } - + def newHiveCompatibleMetastoreTable( + relation: HadoopFsRelation, + serde: HiveSerDe): CatalogTable = { assert(partitionColumns.isEmpty) assert(relation.partitionColumns.isEmpty) - HiveTable( + // TODO We need to place the SQL string in the view text here. + CatalogTable( specifiedDatabase = Option(dbName), name = tblName, - schema = schemaToHiveColumn(relation.schema), - partitionColumns = Nil, - tableType = tableType, - properties = tableProperties.toMap, - serdeProperties = options, - location = Some(relation.paths.head), - viewText = None, // TODO We need to place the SQL string here. - inputFormat = serde.inputFormat, - outputFormat = serde.outputFormat, - serde = serde.serde) + tableType = tableType.toString, + schema = relation.schema.map { f => CatalogColumn(f.name, f.dataType) }, + storage = CatalogStorageFormat( + locationUri = Some(relation.paths.head), + inputFormat = serde.inputFormat, + outputFormat = serde.outputFormat, + serde = serde.serde, + serdeProperties = options + ), + properties = tableProperties.toMap) } // TODO: Support persisting partitioned data source relations in Hive compatible format @@ -431,7 +431,8 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte case Some(aliasText) => Subquery(aliasText, hive.parseSql(viewText)) } } else { - MetastoreRelation(qualifiedTableName.database, qualifiedTableName.name, alias)(table)(hive) + MetastoreRelation( + qualifiedTableName.database, qualifiedTableName.name, alias)(table, client)(hive) } } @@ -604,16 +605,12 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val schema = if (table.schema.nonEmpty) { table.schema } else { - child.output.map { - attr => new HiveColumn( - attr.name, - HiveMetastoreTypes.toMetastoreType(attr.dataType), null) - } + child.output.map { attr => CatalogColumn(attr.name, attr.dataType, attr.nullable) } } val desc = table.copy(schema = schema) - if (hive.convertCTAS && table.serde.isEmpty) { + 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) { @@ -634,10 +631,11 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte child ) } else { - val desc = if (table.serde.isEmpty) { + val desc = if (table.storage.serde.isEmpty) { // add default serde - table.copy( - serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + table.copy(storage = + table.storage.copy(serde = + Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"))) } else { table } @@ -746,9 +744,11 @@ private[hive] case class InsertIntoHiveTable( } } -private[hive] case class MetastoreRelation - (databaseName: String, tableName: String, alias: Option[String]) - (val table: HiveTable) +private[hive] case class MetastoreRelation( + databaseName: String, + tableName: String, + alias: Option[String]) + (val table: CatalogTable, client: HiveClient) (@transient private val sqlContext: SQLContext) extends LeafNode with MultiInstanceRelation with FileRelation { @@ -767,7 +767,11 @@ private[hive] case class MetastoreRelation override protected def otherCopyArgs: Seq[AnyRef] = table :: sqlContext :: Nil - @transient val hiveQlTable: Table = { + private def toHiveColumn(c: CatalogColumn): FieldSchema = { + new FieldSchema(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment.orNull) + } + + @transient val hiveQlTable: HiveTable = { // 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() @@ -782,23 +786,22 @@ private[hive] case class MetastoreRelation val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tTable.setSd(sd) - sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - tTable.setPartitionKeys( - table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) + sd.setCols(table.schema.map(toHiveColumn).asJava) + tTable.setPartitionKeys(table.partitionColumns.map(toHiveColumn).asJava) - table.location.foreach(sd.setLocation) - table.inputFormat.foreach(sd.setInputFormat) - table.outputFormat.foreach(sd.setOutputFormat) + table.storage.locationUri.foreach(sd.setLocation) + table.storage.inputFormat.foreach(sd.setInputFormat) + table.storage.outputFormat.foreach(sd.setOutputFormat) val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo - table.serde.foreach(serdeInfo.setSerializationLib) + table.storage.serde.foreach(serdeInfo.setSerializationLib) sd.setSerdeInfo(serdeInfo) val serdeParameters = new java.util.HashMap[String, String]() - table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + table.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } serdeInfo.setParameters(serdeParameters) - new Table(tTable) + new HiveTable(tTable) } @transient override lazy val statistics: Statistics = Statistics( @@ -823,11 +826,11 @@ private[hive] case class MetastoreRelation // When metastore partition pruning is turned off, we cache the list of all partitions to // mimic the behavior of Spark < 1.5 - lazy val allPartitions = table.getAllPartitions + private lazy val allPartitions: Seq[CatalogTablePartition] = client.getAllPartitions(table) def getHiveQlPartitions(predicates: Seq[Expression] = Nil): Seq[Partition] = { val rawPartitions = if (sqlContext.conf.metastorePartitionPruning) { - table.getPartitions(predicates) + client.getPartitionsByFilter(table, predicates) } else { allPartitions } @@ -836,23 +839,22 @@ private[hive] case class MetastoreRelation val tPartition = new org.apache.hadoop.hive.metastore.api.Partition tPartition.setDbName(databaseName) tPartition.setTableName(tableName) - tPartition.setValues(p.values.asJava) + tPartition.setValues(p.spec.asJava) val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tPartition.setSd(sd) - sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - - sd.setLocation(p.storage.location) - sd.setInputFormat(p.storage.inputFormat) - sd.setOutputFormat(p.storage.outputFormat) + sd.setCols(table.schema.map(toHiveColumn).asJava) + p.storage.locationUri.foreach(sd.setLocation) + p.storage.inputFormat.foreach(sd.setInputFormat) + p.storage.outputFormat.foreach(sd.setOutputFormat) val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo sd.setSerdeInfo(serdeInfo) // maps and lists should be set only after all elements are ready (see HIVE-7975) - serdeInfo.setSerializationLib(p.storage.serde) + p.storage.serde.foreach(serdeInfo.setSerializationLib) val serdeParameters = new java.util.HashMap[String, String]() - table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + table.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } p.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } serdeInfo.setParameters(serdeParameters) @@ -879,10 +881,10 @@ private[hive] case class MetastoreRelation hiveQlTable.getMetadata ) - implicit class SchemaAttribute(f: HiveColumn) { + implicit class SchemaAttribute(f: CatalogColumn) { def toAttribute: AttributeReference = AttributeReference( f.name, - HiveMetastoreTypes.toDataType(f.hiveType), + f.dataType, // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true )(qualifiers = Seq(alias.getOrElse(tableName))) @@ -903,19 +905,22 @@ private[hive] case class MetastoreRelation val columnOrdinals = AttributeMap(attributes.zipWithIndex) override def inputFiles: Array[String] = { - val partLocations = table.getPartitions(Nil).map(_.storage.location).toArray + val partLocations = client + .getPartitionsByFilter(table, Nil) + .flatMap(_.storage.locationUri) + .toArray if (partLocations.nonEmpty) { partLocations } else { Array( - table.location.getOrElse( + table.storage.locationUri.getOrElse( sys.error(s"Could not get the location of ${table.qualifiedName}."))) } } override def newInstance(): MetastoreRelation = { - MetastoreRelation(databaseName, tableName, alias)(table)(sqlContext) + MetastoreRelation(databaseName, tableName, alias)(table, client)(sqlContext) } } 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 1e6bec1527038..02d929ab406c0 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 @@ -32,6 +32,7 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.Logging import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.ParseUtils._ @@ -55,7 +56,7 @@ private[hive] case object NativePlaceholder extends LogicalPlan { } private[hive] case class CreateTableAsSelect( - tableDesc: HiveTable, + tableDesc: CatalogTable, child: LogicalPlan, allowExisting: Boolean) extends UnaryNode with Command { @@ -63,14 +64,14 @@ private[hive] case class CreateTableAsSelect( override lazy val resolved: Boolean = tableDesc.specifiedDatabase.isDefined && tableDesc.schema.nonEmpty && - tableDesc.serde.isDefined && - tableDesc.inputFormat.isDefined && - tableDesc.outputFormat.isDefined && + tableDesc.storage.serde.isDefined && + tableDesc.storage.inputFormat.isDefined && + tableDesc.storage.outputFormat.isDefined && childrenResolved } private[hive] case class CreateViewAsSelect( - tableDesc: HiveTable, + tableDesc: CatalogTable, child: LogicalPlan, allowExisting: Boolean, replace: Boolean, @@ -193,7 +194,7 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging view: ASTNode, viewNameParts: ASTNode, query: ASTNode, - schema: Seq[HiveColumn], + schema: Seq[CatalogColumn], properties: Map[String, String], allowExist: Boolean, replace: Boolean): CreateViewAsSelect = { @@ -201,18 +202,20 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging val originalText = query.source - val tableDesc = HiveTable( + val tableDesc = CatalogTable( specifiedDatabase = dbName, name = viewName, + tableType = TableType.VirtualView.toString, schema = schema, - partitionColumns = Seq.empty[HiveColumn], + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + serdeProperties = Map.empty[String, String] + ), properties = properties, - serdeProperties = Map[String, String](), - tableType = TableType.VirtualView, - location = None, - inputFormat = None, - outputFormat = None, - serde = None, + viewOriginalText = Some(originalText), viewText = Some(originalText)) // We need to keep the original SQL string so that if `spark.sql.nativeView` is @@ -314,8 +317,8 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging val schema = maybeColumns.map { cols => // We can't specify column types when create view, so fill it with null first, and // update it after the schema has been resolved later. - nodeToColumns(cols, lowerCase = true).map(_.copy(hiveType = null)) - }.getOrElse(Seq.empty[HiveColumn]) + nodeToColumns(cols, lowerCase = true).map(_.copy(dataType = null)) + }.getOrElse(Seq.empty[CatalogColumn]) val properties = scala.collection.mutable.Map.empty[String, String] @@ -369,24 +372,23 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging val TableIdentifier(tblName, dbName) = extractTableIdent(tableNameParts) // TODO add bucket support - var tableDesc: HiveTable = HiveTable( + var tableDesc: CatalogTable = CatalogTable( specifiedDatabase = dbName, name = tblName, - schema = Seq.empty[HiveColumn], - partitionColumns = Seq.empty[HiveColumn], - properties = Map[String, String](), - serdeProperties = Map[String, String](), tableType = if (externalTable.isDefined) { - TableType.ExternalTable + TableType.ExternalTable.toString } else { - TableType.ManagedTable + TableType.ManagedTable.toString }, - location = None, - inputFormat = None, - outputFormat = None, - serde = None, - viewText = None) + schema = Seq.empty[CatalogColumn], + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + serdeProperties = Map.empty[String, String] + )) // default storage type abbreviation (e.g. RCFile, ORC, PARQUET etc.) val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT) @@ -397,9 +399,11 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) } - hiveSerDe.inputFormat.foreach(f => tableDesc = tableDesc.copy(inputFormat = Some(f))) - hiveSerDe.outputFormat.foreach(f => tableDesc = tableDesc.copy(outputFormat = Some(f))) - hiveSerDe.serde.foreach(f => tableDesc = tableDesc.copy(serde = Some(f))) + var storDesc = tableDesc.storage + hiveSerDe.inputFormat.foreach { i => storDesc = storDesc.copy(inputFormat = Some(i)) } + hiveSerDe.outputFormat.foreach { o => storDesc = storDesc.copy(outputFormat = Some(o)) } + hiveSerDe.serde.foreach { s => storDesc = storDesc.copy(serde = Some(s)) } + tableDesc = tableDesc.copy(storage = storDesc) children.collect { case list @ Token("TOK_TABCOLLIST", _) => @@ -446,13 +450,16 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging case _ => assert(false) } tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) + storage = tableDesc.storage.copy( + serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams.asScala)) case Token("TOK_TABLELOCATION", child :: Nil) => val location = EximUtil.relativeToAbsolutePath(hiveConf, unescapeSQLString(child.text)) - tableDesc = tableDesc.copy(location = Option(location)) + tableDesc = tableDesc.copy( + storage = tableDesc.storage.copy(locationUri = Option(location))) case Token("TOK_TABLESERIALIZER", child :: Nil) => tableDesc = tableDesc.copy( - serde = Option(unescapeSQLString(child.children.head.text))) + storage = tableDesc.storage.copy( + serde = Option(unescapeSQLString(child.children.head.text)))) if (child.numChildren == 2) { // This is based on the readProps(..) method in // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java: @@ -464,60 +471,74 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging .orNull (unescapeSQLString(prop), value) }.toMap - tableDesc = tableDesc.copy(serdeProperties = tableDesc.serdeProperties ++ serdeParams) + tableDesc = tableDesc.copy( + storage = tableDesc.storage.copy( + serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams)) } case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) => child.text.toLowerCase(Locale.ENGLISH) match { case "orc" => tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) - if (tableDesc.serde.isEmpty) { + storage = tableDesc.storage.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"))) + if (tableDesc.storage.serde.isEmpty) { tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + storage = tableDesc.storage.copy( + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde"))) } case "parquet" => tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) - if (tableDesc.serde.isEmpty) { + storage = tableDesc.storage.copy( + inputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))) + if (tableDesc.storage.serde.isEmpty) { tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + storage = tableDesc.storage.copy( + serde = + Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))) } case "rcfile" => tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy(serde = - Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + storage = tableDesc.storage.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))) + if (tableDesc.storage.serde.isEmpty) { + tableDesc = tableDesc.copy( + storage = tableDesc.storage.copy( + serde = + Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))) } case "textfile" => tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.mapred.TextInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) + storage = tableDesc.storage.copy( + inputFormat = + Option("org.apache.hadoop.mapred.TextInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))) case "sequencefile" => tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), - outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")) + storage = tableDesc.storage.copy( + inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), + outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat"))) case "avro" => tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat")) - if (tableDesc.serde.isEmpty) { + storage = tableDesc.storage.copy( + inputFormat = + Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"))) + if (tableDesc.storage.serde.isEmpty) { tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) + storage = tableDesc.storage.copy( + serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))) } case _ => @@ -527,12 +548,14 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging case Token("TOK_TABLESERIALIZER", Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => - tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName))) + tableDesc = tableDesc.copy( + storage = tableDesc.storage.copy(serde = Option(unquoteString(serdeName)))) otherProps match { case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ getProperties(list)) + storage = tableDesc.storage.copy( + serdeProperties = tableDesc.storage.serdeProperties ++ getProperties(list))) case _ => } @@ -540,10 +563,11 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) case list @ Token("TOK_TABLEFILEFORMAT", _) => tableDesc = tableDesc.copy( - inputFormat = - Option(unescapeSQLString(list.children.head.text)), - outputFormat = - Option(unescapeSQLString(list.children(1).text))) + storage = tableDesc.storage.copy( + inputFormat = + Option(unescapeSQLString(list.children.head.text)), + outputFormat = + Option(unescapeSQLString(list.children(1).text)))) case Token("TOK_STORAGEHANDLER", _) => throw new AnalysisException( "CREATE TABLE AS SELECT cannot be used for a non-native table") @@ -683,15 +707,16 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging // This is based the getColumns methods in // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java - protected def nodeToColumns(node: ASTNode, lowerCase: Boolean): Seq[HiveColumn] = { + protected def nodeToColumns(node: ASTNode, lowerCase: Boolean): Seq[CatalogColumn] = { node.children.map(_.children).collect { case Token(rawColName, Nil) :: colTypeNode :: comment => val colName = if (!lowerCase) rawColName else rawColName.toLowerCase - HiveColumn( - cleanIdentifier(colName), - nodeToTypeString(colTypeNode), - comment.headOption.map(n => unescapeSQLString(n.text)).orNull) + CatalogColumn( + name = cleanIdentifier(colName), + dataType = HiveMetastoreTypes.toDataType(nodeToTypeString(colTypeNode)), + nullable = true, + comment.headOption.map(n => unescapeSQLString(n.text))) } } 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 e061d5b3203e9..9d314f6afa433 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 @@ -19,13 +19,13 @@ package org.apache.spark.sql.hive.client import java.io.PrintStream import java.util.{Map => JMap} -import javax.annotation.Nullable import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} -import org.apache.spark.sql.catalyst.catalog.Database +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTablePartition} import org.apache.spark.sql.catalyst.expressions.Expression +// TODO(andrew): don't do enums. They're gross. private[hive] object TableType extends Enumeration { type TableType = Value val ExternalTable = Value("EXTERNAL_TABLE") @@ -34,53 +34,6 @@ private[hive] object TableType extends Enumeration { val VirtualView = Value("VIRTUAL_VIEW") } -// TODO: Use this for Tables and Partitions -private[hive] case class HiveStorageDescriptor( - location: String, - inputFormat: String, - outputFormat: String, - serde: String, - serdeProperties: Map[String, String]) - -private[hive] case class HivePartition( - values: Seq[String], - storage: HiveStorageDescriptor) - -private[hive] case class HiveColumn(name: String, @Nullable hiveType: String, comment: String) -private[hive] case class HiveTable( - specifiedDatabase: Option[String], - name: String, - schema: Seq[HiveColumn], - partitionColumns: Seq[HiveColumn], - properties: Map[String, String], - serdeProperties: Map[String, String], - tableType: TableType.Value, - location: Option[String] = None, - inputFormat: Option[String] = None, - outputFormat: Option[String] = None, - serde: Option[String] = None, - viewText: Option[String] = None) { - - @transient - private[client] var client: HiveClient = _ - - private[client] def withClient(ci: HiveClient): this.type = { - client = ci - this - } - - def database: String = specifiedDatabase.getOrElse(sys.error("database not resolved")) - - def isPartitioned: Boolean = partitionColumns.nonEmpty - - def getAllPartitions: Seq[HivePartition] = client.getAllPartitions(this) - - def getPartitions(predicates: Seq[Expression]): Seq[HivePartition] = - client.getPartitionsByFilter(this, predicates) - - // Hive does not support backticks when passing names to the client. - def qualifiedName: String = s"$database.$name" -} /** * An externally visible interface to the Hive client. This interface is shared across both the @@ -115,46 +68,46 @@ private[hive] trait HiveClient { def setCurrentDatabase(databaseName: String): Unit /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ - final def getDatabase(name: String): Database = { + final def getDatabase(name: String): CatalogDatabase = { getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException) } /** Returns the metadata for a given database, or None if it doesn't exist. */ - def getDatabaseOption(name: String): Option[Database] + def getDatabaseOption(name: String): Option[CatalogDatabase] /** List the names of all the databases that match the specified pattern. */ def listDatabases(pattern: String): Seq[String] /** Returns the specified table, or throws [[NoSuchTableException]]. */ - def getTable(dbName: String, tableName: String): HiveTable = { + def getTable(dbName: String, tableName: String): CatalogTable = { getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException) } /** Returns the metadata for the specified table or None if it doens't exist. */ - def getTableOption(dbName: String, tableName: String): Option[HiveTable] + def getTableOption(dbName: String, tableName: String): Option[CatalogTable] /** Creates a view with the given metadata. */ - def createView(view: HiveTable): Unit + def createView(view: CatalogTable): Unit /** Updates the given view with new metadata. */ - def alertView(view: HiveTable): Unit + def alertView(view: CatalogTable): Unit /** Creates a table with the given metadata. */ - def createTable(table: HiveTable): Unit + def createTable(table: CatalogTable): Unit /** Drop the specified table. */ def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean): Unit /** Updates the given table with new metadata. */ - final def alterTable(table: HiveTable): Unit = { + final def alterTable(table: CatalogTable): Unit = { alterTable(table.qualifiedName, table) } /** Updates the given table with new metadata, optionally renaming the table. */ - def alterTable(tableName: String, table: HiveTable): Unit + def alterTable(tableName: String, table: CatalogTable): Unit /** Creates a new database with the given name. */ - def createDatabase(database: Database, ignoreIfExists: Boolean): Unit + def createDatabase(database: CatalogDatabase, ignoreIfExists: Boolean): Unit /** * Drop the specified database, if it exists. @@ -168,18 +121,20 @@ private[hive] trait HiveClient { /** * Alter an existing database. This operation does not support renaming. */ - def alterDatabase(name: String, database: Database): Unit + def alterDatabase(name: String, database: CatalogDatabase): Unit /** Returns the specified paritition or None if it does not exist. */ def getPartitionOption( - hTable: HiveTable, - partitionSpec: JMap[String, String]): Option[HivePartition] + hTable: CatalogTable, + partitionSpec: JMap[String, String]): Option[CatalogTablePartition] /** Returns all partitions for the given table. */ - def getAllPartitions(hTable: HiveTable): Seq[HivePartition] + def getAllPartitions(table: CatalogTable): Seq[CatalogTablePartition] /** Returns partitions filtered by predicates for the given table. */ - def getPartitionsByFilter(hTable: HiveTable, predicates: Seq[Expression]): Seq[HivePartition] + def getPartitionsByFilter( + table: CatalogTable, + predicates: Seq[Expression]): Seq[CatalogTablePartition] /** Loads a static partition into an existing table. */ def loadPartition( 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 cd76da3df2150..65b7904150a19 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 @@ -25,10 +25,10 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.{TableType => HTableType} +import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} import org.apache.hadoop.hive.ql.{metadata, Driver} -import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader} @@ -36,9 +36,10 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -import org.apache.spark.sql.catalyst.catalog.Database +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException +import org.apache.spark.sql.hive.HiveMetastoreTypes import org.apache.spark.util.{CircularBuffer, Utils} /** @@ -240,7 +241,7 @@ private[hive] class HiveClientImpl( } override def createDatabase( - database: Database, + database: CatalogDatabase, ignoreIfExists: Boolean): Unit = withHiveState { client.createDatabase( new HiveDatabase( @@ -261,7 +262,7 @@ private[hive] class HiveClientImpl( /** * Alter an existing database. This operation does not support renaming. */ - override def alterDatabase(name: String, database: Database): Unit = withHiveState { + override def alterDatabase(name: String, database: CatalogDatabase): Unit = withHiveState { assert(name == database.name) client.alterDatabase( name, @@ -272,9 +273,9 @@ private[hive] class HiveClientImpl( database.properties.asJava)) } - override def getDatabaseOption(name: String): Option[Database] = withHiveState { + override def getDatabaseOption(name: String): Option[CatalogDatabase] = withHiveState { Option(client.getDatabase(name)).map { d => - Database( + CatalogDatabase( name = d.getName, description = d.getDescription, locationUri = d.getLocationUri, @@ -288,33 +289,36 @@ private[hive] class HiveClientImpl( override def getTableOption( dbName: String, - tableName: String): Option[HiveTable] = withHiveState { - + tableName: String): Option[CatalogTable] = withHiveState { logDebug(s"Looking up $dbName.$tableName") - - val hiveTable = Option(client.getTable(dbName, tableName, false)) - val converted = hiveTable.map { h => - HiveTable( - name = h.getTableName, + Option(client.getTable(dbName, tableName, false)).map { h => + CatalogTable( specifiedDatabase = Option(h.getDbName), - schema = h.getCols.asScala.map(f => HiveColumn(f.getName, f.getType, f.getComment)), - partitionColumns = h.getPartCols.asScala.map(f => - HiveColumn(f.getName, f.getType, f.getComment)), - properties = h.getParameters.asScala.toMap, - serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap, + name = h.getTableName, + // TODO(andrew): make this prettier tableType = h.getTableType match { - case HTableType.MANAGED_TABLE => TableType.ManagedTable - case HTableType.EXTERNAL_TABLE => TableType.ExternalTable - case HTableType.VIRTUAL_VIEW => TableType.VirtualView - case HTableType.INDEX_TABLE => TableType.IndexTable + case HiveTableType.MANAGED_TABLE => TableType.ManagedTable.toString + case HiveTableType.EXTERNAL_TABLE => TableType.ExternalTable.toString + case HiveTableType.VIRTUAL_VIEW => TableType.VirtualView.toString + case HiveTableType.INDEX_TABLE => TableType.IndexTable.toString }, - location = shim.getDataLocation(h), - inputFormat = Option(h.getInputFormatClass).map(_.getName), - outputFormat = Option(h.getOutputFormatClass).map(_.getName), - serde = Option(h.getSerializationLib), - viewText = Option(h.getViewExpandedText)).withClient(this) + schema = h.getCols.asScala.map(fromHiveColumn), + partitionColumns = h.getPartCols.asScala.map(fromHiveColumn), + sortColumns = Seq(), + numBuckets = h.getNumBuckets, + createTime = h.getTTable.getCreateTime.toLong * 1000, + lastAccessTime = h.getLastAccessTime.toLong * 1000, + storage = CatalogStorageFormat( + locationUri = shim.getDataLocation(h), + inputFormat = Option(h.getInputFormatClass).map(_.getName), + outputFormat = Option(h.getOutputFormatClass).map(_.getName), + serde = Option(h.getSerializationLib), + serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap + ), + properties = h.getParameters.asScala.toMap, + viewOriginalText = Option(h.getViewOriginalText), + viewText = Option(h.getViewExpandedText)) } - converted } private def toInputFormat(name: String) = @@ -324,62 +328,63 @@ private[hive] class HiveClientImpl( Utils.classForName(name) .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] - private def toQlTable(table: HiveTable): metadata.Table = { - val qlTable = new metadata.Table(table.database, table.name) - - qlTable.setFields(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - qlTable.setPartCols( - table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - table.properties.foreach { case (k, v) => qlTable.setProperty(k, v) } - table.serdeProperties.foreach { case (k, v) => qlTable.setSerdeParam(k, v) } - - // set owner + private def toHiveColumn(c: CatalogColumn): FieldSchema = { + new FieldSchema(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment.orNull) + } + + private def fromHiveColumn(f: FieldSchema): CatalogColumn = { + new CatalogColumn( + name = f.getName, + dataType = HiveMetastoreTypes.toDataType(f.getType), + nullable = true, + comment = Option(f.getComment)) + } + + private def toQlTable(table: CatalogTable): HiveTable = { + val qlTable = new HiveTable(table.database, table.name) + // TODO(andrew): clean this up + qlTable.setTableType(table.tableType match { + case "MANAGED_TABLE" => HiveTableType.MANAGED_TABLE + case "EXTERNAL_TABLE" => HiveTableType.EXTERNAL_TABLE + case "VIRTUAL_TABLE" => HiveTableType.VIRTUAL_VIEW + case "INDEX_TABLE" => HiveTableType.INDEX_TABLE + }) + qlTable.setFields(table.schema.map(toHiveColumn).asJava) + qlTable.setPartCols(table.partitionColumns.map(toHiveColumn).asJava) + // TODO: set sort columns here too qlTable.setOwner(conf.getUser) - // set create time - qlTable.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - - table.location.foreach { loc => shim.setDataLocation(qlTable, loc) } - table.inputFormat.map(toInputFormat).foreach(qlTable.setInputFormatClass) - table.outputFormat.map(toOutputFormat).foreach(qlTable.setOutputFormatClass) - table.serde.foreach(qlTable.setSerializationLib) - + qlTable.setNumBuckets(table.numBuckets) + qlTable.setCreateTime((table.createTime / 1000).toInt) + qlTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) + table.storage.locationUri.foreach { loc => shim.setDataLocation(qlTable, loc) } + table.storage.inputFormat.map(toInputFormat).foreach(qlTable.setInputFormatClass) + table.storage.outputFormat.map(toOutputFormat).foreach(qlTable.setOutputFormatClass) + table.storage.serde.foreach(qlTable.setSerializationLib) + table.storage.serdeProperties.foreach { case (k, v) => qlTable.setSerdeParam(k, v) } + table.properties.foreach { case (k, v) => qlTable.setProperty(k, v) } + table.viewOriginalText.foreach { t => qlTable.setViewOriginalText(t) } + table.viewText.foreach { t => qlTable.setViewExpandedText(t) } qlTable } - private def toViewTable(view: HiveTable): metadata.Table = { - // TODO: this is duplicated with `toQlTable` except the table type stuff. - val tbl = new metadata.Table(view.database, view.name) - tbl.setTableType(HTableType.VIRTUAL_VIEW) + private def toViewTable(view: CatalogTable): HiveTable = { + val tbl = toQlTable(view) + tbl.setTableType(HiveTableType.VIRTUAL_VIEW) tbl.setSerializationLib(null) tbl.clearSerDeInfo() - - // TODO: we will save the same SQL string to original and expanded text, which is different - // from Hive. - tbl.setViewOriginalText(view.viewText.get) - tbl.setViewExpandedText(view.viewText.get) - - tbl.setFields(view.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - view.properties.foreach { case (k, v) => tbl.setProperty(k, v) } - - // set owner - tbl.setOwner(conf.getUser) - // set create time - tbl.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - tbl } - override def createView(view: HiveTable): Unit = withHiveState { + override def createView(view: CatalogTable): Unit = withHiveState { client.createTable(toViewTable(view)) } - override def alertView(view: HiveTable): Unit = withHiveState { + override def alertView(view: CatalogTable): Unit = withHiveState { client.alterTable(view.qualifiedName, toViewTable(view)) } - override def createTable(table: HiveTable): Unit = withHiveState { - val qlTable = toQlTable(table) - client.createTable(qlTable) + override def createTable(table: CatalogTable): Unit = withHiveState { + client.createTable(toQlTable(table)) } override def dropTable( @@ -389,43 +394,42 @@ private[hive] class HiveClientImpl( client.dropTable(dbName, tableName, true, ignoreIfNotExists) } - override def alterTable(tableName: String, table: HiveTable): Unit = withHiveState { + override def alterTable(tableName: String, table: CatalogTable): Unit = withHiveState { val qlTable = toQlTable(table) val qualifiedTableName = s"${table.database}.$tableName" client.alterTable(qualifiedTableName, qlTable) } - private def toHivePartition(partition: metadata.Partition): HivePartition = { + private def toCatalogPartition(partition: HivePartition): CatalogTablePartition = { val apiPartition = partition.getTPartition - HivePartition( - values = Option(apiPartition.getValues).map(_.asScala).getOrElse(Seq.empty), - storage = HiveStorageDescriptor( - location = apiPartition.getSd.getLocation, - inputFormat = apiPartition.getSd.getInputFormat, - outputFormat = apiPartition.getSd.getOutputFormat, - serde = apiPartition.getSd.getSerdeInfo.getSerializationLib, + CatalogTablePartition( + spec = Option(apiPartition.getValues).map(_.asScala).getOrElse(Seq.empty), + storage = CatalogStorageFormat( + locationUri = Option(apiPartition.getSd.getLocation), + inputFormat = Option(apiPartition.getSd.getInputFormat), + outputFormat = Option(apiPartition.getSd.getOutputFormat), + serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) } override def getPartitionOption( - table: HiveTable, - partitionSpec: JMap[String, String]): Option[HivePartition] = withHiveState { - + table: CatalogTable, + partitionSpec: JMap[String, String]): Option[CatalogTablePartition] = withHiveState { val qlTable = toQlTable(table) val qlPartition = client.getPartition(qlTable, partitionSpec, false) - Option(qlPartition).map(toHivePartition) + Option(qlPartition).map(toCatalogPartition) } - override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState { - val qlTable = toQlTable(hTable) - shim.getAllPartitions(client, qlTable).map(toHivePartition) + override def getAllPartitions(table: CatalogTable): Seq[CatalogTablePartition] = withHiveState { + val qlTable = toQlTable(table) + shim.getAllPartitions(client, qlTable).map(toCatalogPartition) } override def getPartitionsByFilter( - hTable: HiveTable, - predicates: Seq[Expression]): Seq[HivePartition] = withHiveState { - val qlTable = toQlTable(hTable) - shim.getPartitionsByFilter(client, qlTable, predicates).map(toHivePartition) + table: CatalogTable, + predicates: Seq[Expression]): Seq[CatalogTablePartition] = withHiveState { + val qlTable = toQlTable(table) + shim.getPartitionsByFilter(client, qlTable, predicates).map(toCatalogPartition) } override def listTables(dbName: String): Seq[String] = withHiveState { 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 4c0aae6c04bd7..4f838c3141a0c 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 @@ -19,10 +19,10 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes, MetastoreRelation} -import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} /** * Create table and insert the query result into it. @@ -33,7 +33,7 @@ import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} */ private[hive] case class CreateTableAsSelect( - tableDesc: HiveTable, + tableDesc: CatalogTable, query: LogicalPlan, allowExisting: Boolean) extends RunnableCommand { @@ -52,19 +52,19 @@ case class CreateTableAsSelect( val withFormat = tableDesc.copy( - inputFormat = - tableDesc.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), - outputFormat = - tableDesc.outputFormat - .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), - serde = tableDesc.serde.orElse(Some(classOf[LazySimpleSerDe].getName()))) + storage = tableDesc.storage.copy( + inputFormat = + tableDesc.storage.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), + outputFormat = + tableDesc.storage.outputFormat + .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), + serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName())))) val withSchema = if (withFormat.schema.isEmpty) { // Hive doesn't support specifying the column list for target table in CTAS // However we don't think SparkSQL should follow that. - tableDesc.copy(schema = - query.output.map(c => - HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), null))) + tableDesc.copy( + schema = query.output.map { c => CatalogColumn(c.name, c.dataType) }) } else { withFormat } 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 31bda56e8a163..975da94b3455f 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 @@ -19,11 +19,11 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes, SQLBuilder} -import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} +import org.apache.spark.sql.hive.{HiveContext, SQLBuilder} /** * Create Hive view on non-hive-compatible tables by specifying schema ourselves instead of @@ -32,7 +32,7 @@ import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} // TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is different // from Hive and may not work for some cases like create view on self join. private[hive] case class CreateViewAsSelect( - tableDesc: HiveTable, + tableDesc: CatalogTable, child: LogicalPlan, allowExisting: Boolean, orReplace: Boolean) extends RunnableCommand { @@ -70,7 +70,7 @@ private[hive] case class CreateViewAsSelect( Seq.empty[Row] } - private def prepareTable(sqlContext: SQLContext): HiveTable = { + private def prepareTable(sqlContext: SQLContext): CatalogTable = { val expandedText = if (sqlContext.conf.canonicalView) { rebuildViewQueryString(sqlContext).getOrElse(wrapViewTextWithSelect) } else { @@ -79,12 +79,10 @@ private[hive] case class CreateViewAsSelect( val viewSchema = { if (tableDesc.schema.isEmpty) { - childSchema.map { attr => - HiveColumn(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), null) - } + childSchema.map { attr => CatalogColumn(attr.name, attr.dataType) } } else { childSchema.zip(tableDesc.schema).map { case (attr, col) => - HiveColumn(col.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), col.comment) + CatalogColumn(col.name, attr.dataType, nullable = true, col.comment) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 66502bf011a46..f322e2d9c6022 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -83,16 +83,17 @@ class DataSourceWithHiveMetastoreCatalogSuite } val hiveTable = catalog.client.getTable("default", "t") - assert(hiveTable.inputFormat === Some(inputFormat)) - assert(hiveTable.outputFormat === Some(outputFormat)) - assert(hiveTable.serde === Some(serde)) + assert(hiveTable.storage.inputFormat === Some(inputFormat)) + assert(hiveTable.storage.outputFormat === Some(outputFormat)) + assert(hiveTable.storage.serde === Some(serde)) - assert(!hiveTable.isPartitioned) - assert(hiveTable.tableType === ManagedTable) + assert(hiveTable.partitionColumns.isEmpty) + assert(hiveTable.tableType === ManagedTable.toString) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map(_.hiveType) === Seq("decimal(10,3)", "string")) + assert(columns.map { a => HiveMetastoreTypes.toMetastoreType(a.dataType) } === + Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) @@ -114,16 +115,18 @@ class DataSourceWithHiveMetastoreCatalogSuite } val hiveTable = catalog.client.getTable("default", "t") - assert(hiveTable.inputFormat === Some(inputFormat)) - assert(hiveTable.outputFormat === Some(outputFormat)) - assert(hiveTable.serde === Some(serde)) + assert(hiveTable.storage.inputFormat === Some(inputFormat)) + assert(hiveTable.storage.outputFormat === Some(outputFormat)) + assert(hiveTable.storage.serde === Some(serde)) - assert(hiveTable.tableType === ExternalTable) - assert(hiveTable.location.get === path.toURI.toString.stripSuffix(File.separator)) + assert(hiveTable.tableType === ExternalTable.toString) + assert(hiveTable.storage.locationUri === + Some(path.toURI.toString.stripSuffix(File.separator))) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map(_.hiveType) === Seq("decimal(10,3)", "string")) + assert(columns.map { a => HiveMetastoreTypes.toMetastoreType(a.dataType) } === + Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) @@ -143,17 +146,17 @@ class DataSourceWithHiveMetastoreCatalogSuite """.stripMargin) val hiveTable = catalog.client.getTable("default", "t") - assert(hiveTable.inputFormat === Some(inputFormat)) - assert(hiveTable.outputFormat === Some(outputFormat)) - assert(hiveTable.serde === Some(serde)) + assert(hiveTable.storage.inputFormat === Some(inputFormat)) + assert(hiveTable.storage.outputFormat === Some(outputFormat)) + assert(hiveTable.storage.serde === Some(serde)) - assert(hiveTable.isPartitioned === false) - assert(hiveTable.tableType === ExternalTable) - assert(hiveTable.partitionColumns.length === 0) + assert(hiveTable.partitionColumns.isEmpty) + assert(hiveTable.tableType === ExternalTable.toString) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map(_.hiveType) === Seq("int", "string")) + assert(columns.map { a => HiveMetastoreTypes.toMetastoreType(a.dataType) } === + Seq("int", "string")) checkAnswer(table("t"), Row(1, "val_1")) assert(runSqlHive("SELECT * FROM t") === Seq("1\tval_1")) 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 2d1db6ee115bf..2d02014a74716 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 @@ -22,16 +22,17 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.SimpleParserConf import org.apache.spark.sql.catalyst.plans.logical.Generate -import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} import org.apache.spark.sql.hive.client.TableType.{ExternalTable, ManagedTable} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType} class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { val parser = new HiveQl(SimpleParserConf()) - private def extractTableDesc(sql: String): (HiveTable, Boolean) = { + private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { parser.parsePlan(sql).collect { case CreateTableAsSelect(desc, child, allowExisting) => (desc, allowExisting) }.head @@ -54,28 +55,29 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { |AS SELECT * FROM src""".stripMargin val (desc, exists) = extractTableDesc(s1) - assert(exists == true) + assert(exists) assert(desc.specifiedDatabase == Some("mydb")) assert(desc.name == "page_view") - assert(desc.tableType == ExternalTable) - assert(desc.location == Some("/user/external/page_view")) + assert(desc.tableType == ExternalTable.toString) + assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == - HiveColumn("viewtime", "int", null) :: - HiveColumn("userid", "bigint", null) :: - HiveColumn("page_url", "string", null) :: - HiveColumn("referrer_url", "string", null) :: - HiveColumn("ip", "string", "IP Address of the User") :: - HiveColumn("country", "string", "country of origination") :: Nil) + CatalogColumn("viewtime", IntegerType) :: + CatalogColumn("userid", LongType) :: + CatalogColumn("page_url", StringType) :: + CatalogColumn("referrer_url", StringType) :: + CatalogColumn("ip", StringType, comment = Some("IP Address of the User")) :: + CatalogColumn("country", StringType, comment = Some("country of origination")) :: Nil) // TODO will be SQLText assert(desc.viewText == Option("This is the staging page view table")) assert(desc.partitionColumns == - HiveColumn("dt", "string", "date type") :: - HiveColumn("hour", "string", "hour of the day") :: Nil) - assert(desc.serdeProperties == + CatalogColumn("dt", StringType, comment = Some("date type")) :: + CatalogColumn("hour", StringType, comment = Some("hour of the day")) :: Nil) + assert(desc.storage.serdeProperties == Map((serdeConstants.SERIALIZATION_FORMAT, "\054"), (serdeConstants.FIELD_DELIM, "\054"))) - assert(desc.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) - assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - assert(desc.serde == Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + assert(desc.storage.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.storage.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.storage.serde == + Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) } @@ -99,27 +101,27 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { |AS SELECT * FROM src""".stripMargin val (desc, exists) = extractTableDesc(s2) - assert(exists == true) + assert(exists) assert(desc.specifiedDatabase == Some("mydb")) assert(desc.name == "page_view") - assert(desc.tableType == ExternalTable) - assert(desc.location == Some("/user/external/page_view")) + assert(desc.tableType == ExternalTable.toString) + assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == - HiveColumn("viewtime", "int", null) :: - HiveColumn("userid", "bigint", null) :: - HiveColumn("page_url", "string", null) :: - HiveColumn("referrer_url", "string", null) :: - HiveColumn("ip", "string", "IP Address of the User") :: - HiveColumn("country", "string", "country of origination") :: Nil) + CatalogColumn("viewtime", IntegerType) :: + CatalogColumn("userid", LongType) :: + CatalogColumn("page_url", StringType) :: + CatalogColumn("referrer_url", StringType) :: + CatalogColumn("ip", StringType, comment = Some("IP Address of the User")) :: + CatalogColumn("country", StringType, comment = Some("country of origination")) :: Nil) // TODO will be SQLText assert(desc.viewText == Option("This is the staging page view table")) assert(desc.partitionColumns == - HiveColumn("dt", "string", "date type") :: - HiveColumn("hour", "string", "hour of the day") :: Nil) - assert(desc.serdeProperties == Map()) - assert(desc.inputFormat == Option("parquet.hive.DeprecatedParquetInputFormat")) - assert(desc.outputFormat == Option("parquet.hive.DeprecatedParquetOutputFormat")) - assert(desc.serde == Option("parquet.hive.serde.ParquetHiveSerDe")) + CatalogColumn("dt", StringType, comment = Some("date type")) :: + CatalogColumn("hour", StringType, comment = Some("hour of the day")) :: Nil) + assert(desc.storage.serdeProperties == Map()) + assert(desc.storage.inputFormat == Option("parquet.hive.DeprecatedParquetInputFormat")) + assert(desc.storage.outputFormat == Option("parquet.hive.DeprecatedParquetOutputFormat")) + assert(desc.storage.serde == Option("parquet.hive.serde.ParquetHiveSerDe")) assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) } @@ -129,14 +131,15 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(exists == false) assert(desc.specifiedDatabase == None) assert(desc.name == "page_view") - assert(desc.tableType == ManagedTable) - assert(desc.location == None) - assert(desc.schema == Seq.empty[HiveColumn]) + assert(desc.tableType == ManagedTable.toString) + assert(desc.storage.locationUri == None) + assert(desc.schema == Seq.empty[CatalogColumn]) assert(desc.viewText == None) // TODO will be SQLText - assert(desc.serdeProperties == Map()) - assert(desc.inputFormat == Option("org.apache.hadoop.mapred.TextInputFormat")) - assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) - assert(desc.serde.isEmpty) + assert(desc.storage.serdeProperties == Map()) + assert(desc.storage.inputFormat == Option("org.apache.hadoop.mapred.TextInputFormat")) + assert(desc.storage.outputFormat == + Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) + assert(desc.storage.serde.isEmpty) assert(desc.properties == Map()) } @@ -163,14 +166,14 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(exists == false) assert(desc.specifiedDatabase == None) assert(desc.name == "ctas2") - assert(desc.tableType == ManagedTable) - assert(desc.location == None) - assert(desc.schema == Seq.empty[HiveColumn]) + assert(desc.tableType == ManagedTable.toString) + assert(desc.storage.locationUri == None) + assert(desc.schema == Seq.empty[CatalogColumn]) assert(desc.viewText == None) // TODO will be SQLText - assert(desc.serdeProperties == Map(("serde_p1" -> "p1"), ("serde_p2" -> "p2"))) - assert(desc.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) - assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - assert(desc.serde == Option("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) + assert(desc.storage.serdeProperties == Map(("serde_p1" -> "p1"), ("serde_p2" -> "p2"))) + assert(desc.storage.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.storage.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.storage.serde == Option("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) assert(desc.properties == Map(("tbl_p1" -> "p11"), ("tbl_p2" -> "p22"))) } 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 546b3ed5b87d5..07b6084044e67 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 @@ -25,9 +25,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation -import org.apache.spark.sql.hive.client.HiveTable import org.apache.spark.sql.hive.client.TableType.ManagedTable import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -725,18 +725,23 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val tableName = "spark6655" withTable(tableName) { val schema = StructType(StructField("int", IntegerType, true) :: Nil) - val hiveTable = HiveTable( + val hiveTable = CatalogTable( specifiedDatabase = Some("default"), name = tableName, + tableType = ManagedTable.toString, schema = Seq.empty, - partitionColumns = Seq.empty, + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + serdeProperties = Map( + "path" -> catalog.hiveDefaultTableFilePath(TableIdentifier(tableName))) + ), properties = Map( "spark.sql.sources.provider" -> "json", "spark.sql.sources.schema" -> schema.json, - "EXTERNAL" -> "FALSE"), - tableType = ManagedTable, - serdeProperties = Map( - "path" -> catalog.hiveDefaultTableFilePath(TableIdentifier(tableName)))) + "EXTERNAL" -> "FALSE")) catalog.client.createTable(hiveTable) @@ -917,7 +922,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in Hive compatible format, we verify that // each column of the table is of native type StringType. assert(catalog.client.getTable("default", "not_skip_hive_metadata").schema - .forall(column => HiveMetastoreTypes.toDataType(column.hiveType) == StringType)) + .forall(_.dataType == StringType)) catalog.createDataSourceTable( tableIdent = TableIdentifier("skip_hive_metadata"), @@ -931,6 +936,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in SparkSQL format, we verify that // the table has a column type as array of StringType. assert(catalog.client.getTable("default", "skip_hive_metadata").schema - .forall(column => HiveMetastoreTypes.toDataType(column.hiveType) == ArrayType(StringType))) + .forall(_.dataType == ArrayType(StringType))) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index b5f65c96aff2e..488f298981c86 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -28,7 +28,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle val metastoreTable = hiveContext.catalog.client.getTable(dbName, tableName) val expectedPath = hiveContext.catalog.client.getDatabase(dbName).locationUri + "/" + tableName - assert(metastoreTable.serdeProperties("path") === expectedPath) + assert(metastoreTable.storage.serdeProperties("path") === expectedPath) } test(s"saveAsTable() to non-default database - with USE - Overwrite") { 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 5392e6d565052..d17416d559673 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,7 +22,7 @@ import java.io.File import org.apache.hadoop.util.VersionInfo import org.apache.spark.{Logging, SparkFunSuite} -import org.apache.spark.sql.catalyst.catalog.Database +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 import org.apache.spark.sql.hive.HiveContext @@ -61,7 +61,7 @@ class VersionsSuite extends SparkFunSuite with Logging { hadoopVersion = VersionInfo.getVersion, config = buildConf(), ivyPath = ivyPath).createClient() - val db = new Database("default", "desc", "loc", Map()) + val db = new CatalogDatabase("default", "desc", "loc", Map()) badClient.createDatabase(db, ignoreIfExists = true) } @@ -117,27 +117,25 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: createDatabase") { - val db = Database("default", "desc", "loc", Map()) + val db = CatalogDatabase("default", "desc", "loc", Map()) client.createDatabase(db, ignoreIfExists = true) } test(s"$version: createTable") { val table = - HiveTable( + CatalogTable( specifiedDatabase = Option("default"), name = "src", - schema = Seq(HiveColumn("key", "int", "")), - partitionColumns = Seq.empty, - properties = Map.empty, - serdeProperties = Map.empty, - tableType = TableType.ManagedTable, - location = None, - inputFormat = - Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), - outputFormat = - Some(classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), - serde = - Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName())) + tableType = TableType.ManagedTable.toString, + schema = Seq(CatalogColumn("key", IntegerType)), + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), + outputFormat = Some( + classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), + serde = Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName()), + serdeProperties = Map.empty + )) client.createTable(table) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index b91248bfb3fc0..37c01792d9c3f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -149,7 +149,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => val columnNames = columns.map(_.name) - val partValues = if (relation.table.isPartitioned) { + val partValues = if (relation.table.partitionColumns.nonEmpty) { p.prunePartitions(relation.getHiveQlPartitions()).map(_.getValues) } else { Seq.empty From af5ffc0ee84f3dc3c2b9249228293ae7285f916e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 11 Feb 2016 17:34:24 -0800 Subject: [PATCH 12/30] Remove TableType enum Instead, this commit introduces CatalogTableType that serves the same purpose. This adds some type-safety and keeps the code clean. --- .../spark/sql/catalyst/catalog/interface.scala | 18 +++++++++++++++--- .../catalyst/catalog/CatalogTestCases.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 10 +++++----- .../org/apache/spark/sql/hive/HiveQl.scala | 8 ++++---- .../spark/sql/hive/client/HiveClient.scala | 10 ---------- .../spark/sql/hive/client/HiveClientImpl.scala | 18 ++++++++---------- .../sql/hive/HiveMetastoreCatalogSuite.scala | 8 ++++---- .../apache/spark/sql/hive/HiveQlSuite.scala | 11 +++++------ .../sql/hive/MetastoreDataSourcesSuite.scala | 5 ++--- .../spark/sql/hive/client/VersionsSuite.scala | 2 +- 10 files changed, 45 insertions(+), 47 deletions(-) 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 f640d8980f036..9519ec13cd65c 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 @@ -172,7 +172,7 @@ case class CatalogTablePartition(spec: Catalog.TablePartitionSpec, storage: Cata case class CatalogTable( specifiedDatabase: Option[String], name: String, - tableType: String, + tableType: CatalogTableType, storage: CatalogStorageFormat, schema: Seq[CatalogColumn], partitionColumns: Seq[CatalogColumn] = Seq.empty, @@ -184,8 +184,11 @@ case class CatalogTable( viewOriginalText: Option[String] = None, viewText: Option[String] = None) { - require(tableType == "EXTERNAL_TABLE" || tableType == "INDEX_TABLE" || - tableType == "MANAGED_TABLE" || tableType == "VIRTUAL_VIEW") + require( + tableType == CatalogTableType.EXTERNAL_TABLE || + tableType == CatalogTableType.INDEX_TABLE || + tableType == CatalogTableType.MANAGED_TABLE || + tableType == CatalogTableType.VIRTUAL_VIEW) /** Return the database this table was specified to belong to, assuming it exists. */ def database: String = specifiedDatabase.getOrElse(sys.error("database not resolved")) @@ -195,6 +198,15 @@ case class CatalogTable( } +class CatalogTableType private(name: String) +object CatalogTableType { + val EXTERNAL_TABLE = new CatalogTableType("EXTERNAL_TABLE") + val MANAGED_TABLE = new CatalogTableType("MANAGED_TABLE") + val INDEX_TABLE = new CatalogTableType("INDEX_TABLE") + val VIRTUAL_VIEW = new CatalogTableType("VIRTUAL_VIEW") +} + + /** * A database defined in the catalog. */ 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 a217b120c9084..9595011ebbcea 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 @@ -66,7 +66,7 @@ abstract class CatalogTestCases extends SparkFunSuite { CatalogTable( specifiedDatabase = Some("db"), name = name, - tableType = "EXTERNAL_TABLE", + tableType = CatalogTableType.EXTERNAL_TABLE, storage = CatalogStorageFormat( locationUri = Some("just works"), inputFormat = Some("this part will change"), 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 b69a3dda3de3d..d8c865628bfef 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 @@ -279,10 +279,10 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val tableType = if (isExternal) { tableProperties.put("EXTERNAL", "TRUE") - TableType.ExternalTable + CatalogTableType.EXTERNAL_TABLE } else { tableProperties.put("EXTERNAL", "FALSE") - TableType.ManagedTable + CatalogTableType.MANAGED_TABLE } val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.hiveconf) @@ -293,7 +293,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte CatalogTable( specifiedDatabase = Option(dbName), name = tblName, - tableType = tableType.toString, + tableType = tableType, schema = Nil, storage = CatalogStorageFormat( locationUri = None, @@ -315,7 +315,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte CatalogTable( specifiedDatabase = Option(dbName), name = tblName, - tableType = tableType.toString, + tableType = tableType, schema = relation.schema.map { f => CatalogColumn(f.name, f.dataType) }, storage = CatalogStorageFormat( locationUri = Some(relation.paths.head), @@ -422,7 +422,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte // Then, if alias is specified, wrap the table with a Subquery using the alias. // Otherwise, wrap the table with a Subquery using the table name. alias.map(a => Subquery(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) - } else if (table.tableType == TableType.VirtualView) { + } else if (table.tableType == CatalogTableType.VIRTUAL_VIEW) { val viewText = table.viewText.getOrElse(sys.error("Invalid view without text.")) alias match { // because hive use things like `_c0` to build the expanded text 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 02d929ab406c0..dbe02e02b89ea 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 @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.Logging import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.ParseUtils._ @@ -205,7 +205,7 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging val tableDesc = CatalogTable( specifiedDatabase = dbName, name = viewName, - tableType = TableType.VirtualView.toString, + tableType = CatalogTableType.VIRTUAL_VIEW, schema = schema, storage = CatalogStorageFormat( locationUri = None, @@ -377,9 +377,9 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging name = tblName, tableType = if (externalTable.isDefined) { - TableType.ExternalTable.toString + CatalogTableType.EXTERNAL_TABLE } else { - TableType.ManagedTable.toString + CatalogTableType.MANAGED_TABLE }, schema = Seq.empty[CatalogColumn], storage = CatalogStorageFormat( 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 9d314f6afa433..8eb2e32a809fc 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 @@ -25,16 +25,6 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, Cat import org.apache.spark.sql.catalyst.expressions.Expression -// TODO(andrew): don't do enums. They're gross. -private[hive] object TableType extends Enumeration { - type TableType = Value - val ExternalTable = Value("EXTERNAL_TABLE") - val IndexTable = Value("INDEX_TABLE") - val ManagedTable = Value("MANAGED_TABLE") - val VirtualView = Value("VIRTUAL_VIEW") -} - - /** * An externally visible interface to the Hive client. This interface is shared across both the * internal and external classloaders for a given version of Hive and thus must expose only 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 65b7904150a19..dfdf89939af73 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 @@ -295,12 +295,11 @@ private[hive] class HiveClientImpl( CatalogTable( specifiedDatabase = Option(h.getDbName), name = h.getTableName, - // TODO(andrew): make this prettier tableType = h.getTableType match { - case HiveTableType.MANAGED_TABLE => TableType.ManagedTable.toString - case HiveTableType.EXTERNAL_TABLE => TableType.ExternalTable.toString - case HiveTableType.VIRTUAL_VIEW => TableType.VirtualView.toString - case HiveTableType.INDEX_TABLE => TableType.IndexTable.toString + case HiveTableType.EXTERNAL_TABLE => CatalogTableType.EXTERNAL_TABLE + case HiveTableType.MANAGED_TABLE => CatalogTableType.MANAGED_TABLE + case HiveTableType.INDEX_TABLE => CatalogTableType.INDEX_TABLE + case HiveTableType.VIRTUAL_VIEW => CatalogTableType.VIRTUAL_VIEW }, schema = h.getCols.asScala.map(fromHiveColumn), partitionColumns = h.getPartCols.asScala.map(fromHiveColumn), @@ -342,12 +341,11 @@ private[hive] class HiveClientImpl( private def toQlTable(table: CatalogTable): HiveTable = { val qlTable = new HiveTable(table.database, table.name) - // TODO(andrew): clean this up qlTable.setTableType(table.tableType match { - case "MANAGED_TABLE" => HiveTableType.MANAGED_TABLE - case "EXTERNAL_TABLE" => HiveTableType.EXTERNAL_TABLE - case "VIRTUAL_TABLE" => HiveTableType.VIRTUAL_VIEW - case "INDEX_TABLE" => HiveTableType.INDEX_TABLE + case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE + case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE + case CatalogTableType.INDEX_TABLE => HiveTableType.INDEX_TABLE + case CatalogTableType.VIRTUAL_VIEW => HiveTableType.VIRTUAL_VIEW }) qlTable.setFields(table.schema.map(toHiveColumn).asJava) qlTable.setPartCols(table.partitionColumns.map(toHiveColumn).asJava) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index f322e2d9c6022..87d61d8e93a95 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -21,7 +21,7 @@ import java.io.File import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{QueryTest, Row, SaveMode, SQLConf} -import org.apache.spark.sql.hive.client.TableType.{ExternalTable, ManagedTable} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} import org.apache.spark.sql.types.{DecimalType, StringType, StructType} @@ -88,7 +88,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(hiveTable.storage.serde === Some(serde)) assert(hiveTable.partitionColumns.isEmpty) - assert(hiveTable.tableType === ManagedTable.toString) + assert(hiveTable.tableType === CatalogTableType.MANAGED_TABLE) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) @@ -119,7 +119,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(hiveTable.storage.outputFormat === Some(outputFormat)) assert(hiveTable.storage.serde === Some(serde)) - assert(hiveTable.tableType === ExternalTable.toString) + assert(hiveTable.tableType === CatalogTableType.EXTERNAL_TABLE) assert(hiveTable.storage.locationUri === Some(path.toURI.toString.stripSuffix(File.separator))) @@ -151,7 +151,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(hiveTable.storage.serde === Some(serde)) assert(hiveTable.partitionColumns.isEmpty) - assert(hiveTable.tableType === ExternalTable.toString) + assert(hiveTable.tableType === CatalogTableType.EXTERNAL_TABLE) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) 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 2d02014a74716..d193ea115166d 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 @@ -22,11 +22,10 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.SimpleParserConf import org.apache.spark.sql.catalyst.plans.logical.Generate -import org.apache.spark.sql.hive.client.TableType.{ExternalTable, ManagedTable} import org.apache.spark.sql.types.{IntegerType, LongType, StringType} class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { @@ -58,7 +57,7 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(exists) assert(desc.specifiedDatabase == Some("mydb")) assert(desc.name == "page_view") - assert(desc.tableType == ExternalTable.toString) + assert(desc.tableType == CatalogTableType.EXTERNAL_TABLE) assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == CatalogColumn("viewtime", IntegerType) :: @@ -104,7 +103,7 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(exists) assert(desc.specifiedDatabase == Some("mydb")) assert(desc.name == "page_view") - assert(desc.tableType == ExternalTable.toString) + assert(desc.tableType == CatalogTableType.EXTERNAL_TABLE) assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == CatalogColumn("viewtime", IntegerType) :: @@ -131,7 +130,7 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(exists == false) assert(desc.specifiedDatabase == None) assert(desc.name == "page_view") - assert(desc.tableType == ManagedTable.toString) + assert(desc.tableType == CatalogTableType.MANAGED_TABLE) assert(desc.storage.locationUri == None) assert(desc.schema == Seq.empty[CatalogColumn]) assert(desc.viewText == None) // TODO will be SQLText @@ -166,7 +165,7 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(exists == false) assert(desc.specifiedDatabase == None) assert(desc.name == "ctas2") - assert(desc.tableType == ManagedTable.toString) + assert(desc.tableType == CatalogTableType.MANAGED_TABLE) assert(desc.storage.locationUri == None) assert(desc.schema == Seq.empty[CatalogColumn]) assert(desc.viewText == None) // TODO will be SQLText 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 07b6084044e67..5591d024cbefc 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 @@ -25,10 +25,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation -import org.apache.spark.sql.hive.client.TableType.ManagedTable import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ @@ -728,7 +727,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val hiveTable = CatalogTable( specifiedDatabase = Some("default"), name = tableName, - tableType = ManagedTable.toString, + tableType = CatalogTableType.MANAGED_TABLE, schema = Seq.empty, storage = CatalogStorageFormat( locationUri = None, 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 d17416d559673..74a08bdfe6682 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 @@ -126,7 +126,7 @@ class VersionsSuite extends SparkFunSuite with Logging { CatalogTable( specifiedDatabase = Option("default"), name = "src", - tableType = TableType.ManagedTable.toString, + tableType = CatalogTableType.MANAGED_TABLE, schema = Seq(CatalogColumn("key", IntegerType)), storage = CatalogStorageFormat( locationUri = None, From d7b18e628374659f0a792d5c5a9154711fc9073b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 11 Feb 2016 17:48:30 -0800 Subject: [PATCH 13/30] Re-implement all table operations after the refactor --- .../catalyst/catalog/InMemoryCatalog.scala | 10 ++---- .../sql/catalyst/catalog/interface.scala | 5 +-- .../catalyst/catalog/CatalogTestCases.scala | 6 ++-- .../apache/spark/sql/hive/HiveCatalog.scala | 34 +++++++++++-------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++-- .../spark/sql/hive/client/HiveClient.scala | 5 ++- .../sql/hive/client/HiveClientImpl.scala | 8 +++-- .../hive/execution/CreateTableAsSelect.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../spark/sql/hive/client/VersionsSuite.scala | 2 +- 10 files changed, 43 insertions(+), 37 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 7fc076fd41336..852c61ac66676 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 @@ -188,13 +188,9 @@ class InMemoryCatalog extends Catalog { catalog(db).tables.remove(oldName) } - override def alterTable( - db: String, - table: String, - tableDefinition: CatalogTable): Unit = synchronized { - assertTableExists(db, table) - assert(table == tableDefinition.name) - catalog(db).tables(table).table = tableDefinition + override def alterTable(db: String, tableDefinition: CatalogTable): Unit = synchronized { + assertTableExists(db, tableDefinition.name) + catalog(db).tables(tableDefinition.name).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 9519ec13cd65c..a5ce55664191b 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 @@ -63,10 +63,7 @@ abstract class Catalog { def renameTable(db: String, oldName: String, newName: String): Unit - /** - * Alter an existing table. This operation does not support renaming. - */ - def alterTable(db: String, table: String, tableDefinition: CatalogTable): Unit + def alterTable(db: String, tableDefinition: CatalogTable): Unit def getTable(db: String, table: String): CatalogTable 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 9595011ebbcea..d19542445ed22 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 @@ -217,7 +217,7 @@ abstract class CatalogTestCases extends SparkFunSuite { test("alter table") { val catalog = newBasicCatalog() - catalog.alterTable("db2", "tbl1", newTable("tbl1").copy(createTime = 10)) + catalog.alterTable("db2", newTable("tbl1").copy(createTime = 10)) assert(catalog.getTable("db2", "tbl1").createTime == 10) } @@ -225,11 +225,11 @@ abstract class CatalogTestCases extends SparkFunSuite { val catalog = newBasicCatalog() intercept[AnalysisException] { // Throw exception when the database does not exist - catalog.alterTable("unknown_db", "unknown_table", newTable("unknown_table")) + catalog.alterTable("unknown_db", newTable("unknown_table")) } intercept[AnalysisException] { // Throw exception when the table does not exist - catalog.alterTable("db2", "unknown_table", newTable("unknown_table")) + catalog.alterTable("db2", newTable("unknown_table")) } } 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 27a4048e370b0..7b0b158fc132e 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 @@ -29,6 +29,14 @@ import org.apache.spark.sql.hive.client.HiveClient private[spark] class HiveCatalog(client: HiveClient) extends Catalog { import Catalog._ + /** + * Assert that the provided database matches the one specified in the table. + */ + private def assertDbMatches(db: String, table: CatalogTable): Unit = { + assert(table.specifiedDatabase == Some(db), + "provided database does not much the one specified in the table definition") + } + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -77,40 +85,38 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + assertDbMatches(db, tableDefinition) + client.createTable(tableDefinition, ignoreIfExists) } override def dropTable( db: String, table: String, ignoreIfNotExists: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + client.dropTable(db, table, ignoreIfNotExists) } override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { - throw new UnsupportedOperationException + val newTable = client.getTable(db, oldName).copy(name = newName) + assertDbMatches(db, newTable) + client.alterTable(oldName, newTable) } - /** - * Alter an existing table. This operation does not support renaming. - */ - override def alterTable( - db: String, - table: String, - tableDefinition: CatalogTable): Unit = synchronized { - throw new UnsupportedOperationException + override def alterTable(db: String, tableDefinition: CatalogTable): Unit = synchronized { + assertDbMatches(db, tableDefinition) + client.alterTable(tableDefinition) } override def getTable(db: String, table: String): CatalogTable = synchronized { - throw new UnsupportedOperationException + client.getTable(db, table) } override def listTables(db: String): Seq[String] = synchronized { - throw new UnsupportedOperationException + client.listTables(db) } override def listTables(db: String, pattern: String): Seq[String] = synchronized { - throw new UnsupportedOperationException + client.listTables(db, pattern) } // -------------------------------------------------------------------------- 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 d8c865628bfef..7ee10d4c55cb9 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 @@ -381,7 +381,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte // specific way. try { logInfo(message) - client.createTable(table) + client.createTable(table, ignoreIfExists = false) } catch { case throwable: Throwable => val warningMessage = @@ -389,13 +389,13 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte s"it into Hive metastore in Spark SQL specific format." logWarning(warningMessage, throwable) val sparkSqlSpecificTable = newSparkSQLSpecificMetastoreTable() - client.createTable(sparkSqlSpecificTable) + client.createTable(sparkSqlSpecificTable, ignoreIfExists = false) } case (None, message) => logWarning(message) val hiveTable = newSparkSQLSpecificMetastoreTable() - client.createTable(hiveTable) + client.createTable(hiveTable, ignoreIfExists = false) } } 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 8eb2e32a809fc..d6171525d55dd 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 @@ -51,6 +51,9 @@ private[hive] trait HiveClient { /** Returns the names of all tables in the given database. */ def listTables(dbName: String): Seq[String] + /** Returns the names of tables in the given database that matches the given pattern. */ + def listTables(dbName: String, pattern: String): Seq[String] + /** Returns the name of the active database. */ def currentDatabase: String @@ -83,7 +86,7 @@ private[hive] trait HiveClient { def alertView(view: CatalogTable): Unit /** Creates a table with the given metadata. */ - def createTable(table: CatalogTable): Unit + def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit /** Drop the specified table. */ def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean): 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 dfdf89939af73..99fac71412f0d 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 @@ -381,8 +381,8 @@ private[hive] class HiveClientImpl( client.alterTable(view.qualifiedName, toViewTable(view)) } - override def createTable(table: CatalogTable): Unit = withHiveState { - client.createTable(toQlTable(table)) + override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState { + client.createTable(toQlTable(table), ignoreIfExists) } override def dropTable( @@ -434,6 +434,10 @@ private[hive] class HiveClientImpl( client.getAllTables(dbName).asScala } + override def listTables(dbName: String, pattern: String): Seq[String] = withHiveState { + client.getTablesByPattern(dbName, pattern).asScala + } + /** * Runs the specified SQL query using Hive. */ 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 4f838c3141a0c..ce975bb44fbc1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -69,7 +69,7 @@ case class CreateTableAsSelect( withFormat } - hiveContext.catalog.client.createTable(withSchema) + hiveContext.catalog.client.createTable(withSchema, ignoreIfExists = false) // Get the Metastore Relation hiveContext.catalog.lookupRelation(tableIdentifier, None) match { 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 5591d024cbefc..a9c5104a6b5e2 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 @@ -742,7 +742,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv "spark.sql.sources.schema" -> schema.json, "EXTERNAL" -> "FALSE")) - catalog.client.createTable(hiveTable) + catalog.client.createTable(hiveTable, ignoreIfExists = false) invalidateTable(tableName) val actualSchema = table(tableName).schema 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 74a08bdfe6682..2b8548ed339cc 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 @@ -137,7 +137,7 @@ class VersionsSuite extends SparkFunSuite with Logging { serdeProperties = Map.empty )) - client.createTable(table) + client.createTable(table, ignoreIfExists = false) } test(s"$version: getTable") { From a915d01eac651994c4d69b961299b476fe40f77d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 12 Feb 2016 12:50:39 -0800 Subject: [PATCH 14/30] Implement all partition operations --- .../spark/sql/catalyst/analysis/Catalog.scala | 14 +- .../catalyst/catalog/InMemoryCatalog.scala | 28 ++-- .../sql/catalyst/catalog/interface.scala | 19 ++- .../catalyst/catalog/CatalogTestCases.scala | 53 ++++++-- .../apache/spark/sql/hive/HiveCatalog.scala | 26 ++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/client/HiveClient.scala | 69 +++++++++- .../sql/hive/client/HiveClientImpl.scala | 128 +++++++++++++----- .../hive/execution/InsertIntoHiveTable.scala | 2 +- 9 files changed, 250 insertions(+), 91 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index f2f9ec59417ef..c2f2581872b82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -20,20 +20,22 @@ package org.apache.spark.sql.catalyst.analysis import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystConf, EmptyConf, TableIdentifier} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} + /** - * Thrown by a catalog when a table cannot be found. The analyzer will rethrow the exception - * as an AnalysisException with the correct position information. + * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the + * exception as an [[AnalysisException]] with the correct position information. */ -class NoSuchTableException extends Exception +abstract class NoSuchItemException extends Exception + +class NoSuchPartitionException extends NoSuchItemException +class NoSuchTableException extends NoSuchItemException +class NoSuchDatabaseException extends NoSuchItemException -class NoSuchDatabaseException extends Exception /** * An interface for looking up relations by name. Used by an [[Analyzer]]. 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 852c61ac66676..c513f0956913c 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 @@ -248,18 +248,28 @@ class InMemoryCatalog extends Catalog { partSpecs.foreach(existingParts.remove) } - override def alterPartition( + override def renamePartitions( db: String, table: String, - spec: TablePartitionSpec, - newPart: CatalogTablePartition): Unit = synchronized { - assertPartitionExists(db, table, spec) - val existingParts = catalog(db).tables(table).partitions - if (spec != newPart.spec) { - // Also a change in specs; remove the old one and add the new one back - existingParts.remove(spec) + specs: Seq[TablePartitionSpec], + newSpecs: Seq[TablePartitionSpec]): Unit = synchronized { + assert(specs.size == newSpecs.size, "number of old and new partition specs differ") + specs.zip(newSpecs).foreach { case (oldSpec, newSpec) => + val newPart = getPartition(db, table, oldSpec).copy(spec = newSpec) + val existingParts = catalog(db).tables(table).partitions + existingParts.remove(oldSpec) + existingParts.put(newSpec, newPart) + } + } + + override def alterPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition]): Unit = synchronized { + parts.foreach { p => + assertPartitionExists(db, table, p.spec) + catalog(db).tables(table).partitions.put(p.spec, p) } - existingParts.put(newPart.spec, newPart) } override def getPartition( 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 a5ce55664191b..3ba627fe073f0 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 @@ -88,13 +88,19 @@ abstract class Catalog { ignoreIfNotExists: Boolean): Unit /** - * Alter an existing table partition and optionally override its spec. + * 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 alterPartition( + def renamePartitions( db: String, table: String, - spec: TablePartitionSpec, - newPart: CatalogTablePartition): Unit + specs: Seq[TablePartitionSpec], + newSpecs: Seq[TablePartitionSpec]): Unit + + def alterPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition]): Unit def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition @@ -215,5 +221,8 @@ case class CatalogDatabase( object Catalog { - type TablePartitionSpec = Seq[String] + /** + * Specifications of a table partition indexed by column name. + */ + type TablePartitionSpec = Map[String, 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 d19542445ed22..55f38630413f5 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 @@ -28,9 +28,9 @@ import org.apache.spark.sql.AnalysisException */ abstract class CatalogTestCases extends SparkFunSuite { private val storageFormat = CatalogStorageFormat(None, Some("z"), Some("y"), Some("x"), Map()) - private val part1 = CatalogTablePartition(Seq("1"), storageFormat) - private val part2 = CatalogTablePartition(Seq("2"), storageFormat) - private val part3 = CatalogTablePartition(Seq("3"), storageFormat) + private val part1 = CatalogTablePartition(Map("a" -> "1"), storageFormat) + private val part2 = CatalogTablePartition(Map("b" -> "2"), storageFormat) + private val part3 = CatalogTablePartition(Map("c" -> "3"), storageFormat) private val funcClass = "org.apache.spark.myFunc" protected def newEmptyCatalog(): Catalog @@ -344,28 +344,53 @@ abstract class CatalogTestCases extends SparkFunSuite { } } + test("rename partitions") { + val catalog = newBasicCatalog() + val newPart1 = part1.copy(spec = Map("x" -> "10")) + val newPart2 = part2.copy(spec = Map("y" -> "12")) + val newSpecs = Seq(newPart1.spec, newPart2.spec) + catalog.renamePartitions("db2", "tbl2", Seq(part1.spec, part2.spec), newSpecs) + assert(catalog.getPartition("db2", "tbl2", newPart1.spec) === newPart1) + assert(catalog.getPartition("db2", "tbl2", newPart2.spec) === newPart2) + intercept[AnalysisException] { catalog.getPartition("db2", "tbl2", part1.spec) } + intercept[AnalysisException] { catalog.getPartition("db2", "tbl2", part2.spec) } + } + + test("rename partitions when database / table does not exist") { + val catalog = newBasicCatalog() + intercept[AnalysisException] { + catalog.renamePartitions("does_not_exist", "tbl1", Seq(part1.spec), Seq(part2.spec)) + } + intercept[AnalysisException] { + catalog.renamePartitions("db2", "does_not_exist", Seq(part1.spec), Seq(part2.spec)) + } + } + test("alter partitions") { val catalog = newBasicCatalog() - val partSameSpec = part1.copy(storage = storageFormat.copy(serde = Some("myserde"))) - val partNewSpec = part1.copy(spec = Seq("10")) + val newPart1 = part1.copy(storage = storageFormat.copy(locationUri = Some("usa"))) + val newPart2 = part2.copy(storage = storageFormat.copy(locationUri = Some("china"))) + assert(catalog.getPartition("db2", "tbl2", part1.spec).storage.locationUri.isEmpty) + assert(catalog.getPartition("db2", "tbl2", part2.spec).storage.locationUri.isEmpty) // alter but keep spec the same - catalog.alterPartition("db2", "tbl2", part1.spec, partSameSpec) - assert(catalog.getPartition("db2", "tbl2", part1.spec) == partSameSpec) - // alter and change spec - catalog.alterPartition("db2", "tbl2", part1.spec, partNewSpec) + catalog.alterPartitions("db2", "tbl2", Seq(newPart1, newPart2)) + assert(catalog.getPartition("db2", "tbl2", part1.spec).storage.locationUri === Some("usa")) + assert(catalog.getPartition("db2", "tbl2", part2.spec).storage.locationUri === Some("china")) + // alter but change spec, should fail because new partition specs do not exist yet + val badPart1 = part1.copy(spec = Map("k" -> "v")) + val badPart2 = part2.copy(spec = Map("k" -> "v")) intercept[AnalysisException] { - catalog.getPartition("db2", "tbl2", part1.spec) + catalog.alterPartitions("db2", "tbl2", Seq(badPart1, badPart2)) } - assert(catalog.getPartition("db2", "tbl2", partNewSpec.spec) == partNewSpec) } - test("alter partition when database / table does not exist") { + test("alter partitions when database / table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { - catalog.alterPartition("does_not_exist", "tbl1", part1.spec, part1) + catalog.alterPartitions("does_not_exist", "tbl1", Seq(part1)) } intercept[AnalysisException] { - catalog.alterPartition("db2", "does_not_exist", part1.spec, part1) + catalog.alterPartitions("db2", "does_not_exist", Seq(part1)) } } 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 7b0b158fc132e..ca4b25e0be26c 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 @@ -128,7 +128,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { table: String, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + client.createPartitions(db, table, parts, ignoreIfExists) } override def dropPartitions( @@ -136,31 +136,35 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { table: String, parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + client.dropPartitions(db, table, parts, ignoreIfNotExists) } - /** - * Alter an existing table partition and optionally override its spec. - */ - override def alterPartition( + override def renamePartitions( db: String, table: String, - spec: TablePartitionSpec, - newPart: CatalogTablePartition): Unit = synchronized { - throw new UnsupportedOperationException + specs: Seq[TablePartitionSpec], + newSpecs: Seq[TablePartitionSpec]): Unit = synchronized { + client.renamePartitions(db, table, specs, newSpecs) + } + + override def alterPartitions( + db: String, + table: String, + newParts: Seq[CatalogTablePartition]): Unit = synchronized { + client.alterPartitions(db, table, newParts) } override def getPartition( db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition = synchronized { - throw new UnsupportedOperationException + client.getPartition(db, table, spec) } override def listPartitions( db: String, table: String): Seq[CatalogTablePartition] = synchronized { - throw new UnsupportedOperationException + client.getAllPartitions(db, table) } // -------------------------------------------------------------------------- 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 7ee10d4c55cb9..80d7e40e338dd 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 @@ -839,7 +839,7 @@ private[hive] case class MetastoreRelation( val tPartition = new org.apache.hadoop.hive.metastore.api.Partition tPartition.setDbName(databaseName) tPartition.setTableName(tableName) - tPartition.setValues(p.spec.asJava) + tPartition.setValues(p.spec.values.toList.asJava) val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tPartition.setSd(sd) 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 d6171525d55dd..4564f15135918 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 @@ -18,10 +18,9 @@ package org.apache.spark.sql.hive.client import java.io.PrintStream -import java.util.{Map => JMap} -import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} -import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTablePartition} +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression @@ -72,7 +71,7 @@ private[hive] trait HiveClient { def listDatabases(pattern: String): Seq[String] /** Returns the specified table, or throws [[NoSuchTableException]]. */ - def getTable(dbName: String, tableName: String): CatalogTable = { + final def getTable(dbName: String, tableName: String): CatalogTable = { getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException) } @@ -116,10 +115,66 @@ private[hive] trait HiveClient { */ def alterDatabase(name: String, database: CatalogDatabase): Unit - /** Returns the specified paritition or None if it does not exist. */ + /** + * Create one or many partitions in the given table. + */ + def createPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit + + /** + * Drop one or many partitions in the given table. + */ + def dropPartitions( + db: String, + table: String, + specs: Seq[Catalog.TablePartitionSpec], + ignoreIfNotExists: Boolean): Unit + + /** + * Rename one or many existing table partitions, assuming they exist. + */ + def renamePartitions( + db: String, + table: String, + specs: Seq[Catalog.TablePartitionSpec], + newSpecs: Seq[Catalog.TablePartitionSpec]): Unit + + /** + * Alter one or many existing table partitions, assuming they exist. + */ + def alterPartitions( + db: String, + table: String, + newParts: Seq[CatalogTablePartition]): Unit + + /** Returns the specified partition, or throws [[NoSuchPartitionException]]. */ + final def getPartition( + dbName: String, + tableName: String, + spec: Catalog.TablePartitionSpec): CatalogTablePartition = { + getPartitionOption(dbName, tableName, spec).getOrElse(throw new NoSuchPartitionException) + } + + /** Returns the specified partition or None if it does not exist. */ + final def getPartitionOption( + db: String, + table: String, + spec: Catalog.TablePartitionSpec): Option[CatalogTablePartition] = { + getPartitionOption(getTable(db, table), spec) + } + + /** Returns the specified partition or None if it does not exist. */ def getPartitionOption( - hTable: CatalogTable, - partitionSpec: JMap[String, String]): Option[CatalogTablePartition] + table: CatalogTable, + spec: Catalog.TablePartitionSpec): Option[CatalogTablePartition] + + /** Returns all partitions for the given table. */ + final def getAllPartitions(db: String, table: String): Seq[CatalogTablePartition] = { + getAllPartitions(getTable(db, table)) + } /** Returns all partitions for the given table. */ def getAllPartitions(table: CatalogTable): Seq[CatalogTablePartition] 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 99fac71412f0d..d63ce4572e342 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 @@ -18,24 +18,25 @@ package org.apache.spark.sql.hive.client import java.io.{File, PrintStream} -import java.util.{Map => JMap} import scala.collection.JavaConverters._ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} +import org.apache.hadoop.hive.metastore.{PartitionDropOptions, TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} -import org.apache.hadoop.hive.ql.{metadata, Driver} +import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} +import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader} import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkConf, SparkException} -import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException @@ -339,34 +340,34 @@ private[hive] class HiveClientImpl( comment = Option(f.getComment)) } - private def toQlTable(table: CatalogTable): HiveTable = { - val qlTable = new HiveTable(table.database, table.name) - qlTable.setTableType(table.tableType match { + private def toHiveTable(table: CatalogTable): HiveTable = { + val hiveTable = new HiveTable(table.database, table.name) + hiveTable.setTableType(table.tableType match { case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE case CatalogTableType.INDEX_TABLE => HiveTableType.INDEX_TABLE case CatalogTableType.VIRTUAL_VIEW => HiveTableType.VIRTUAL_VIEW }) - qlTable.setFields(table.schema.map(toHiveColumn).asJava) - qlTable.setPartCols(table.partitionColumns.map(toHiveColumn).asJava) + hiveTable.setFields(table.schema.map(toHiveColumn).asJava) + hiveTable.setPartCols(table.partitionColumns.map(toHiveColumn).asJava) // TODO: set sort columns here too - qlTable.setOwner(conf.getUser) - qlTable.setNumBuckets(table.numBuckets) - qlTable.setCreateTime((table.createTime / 1000).toInt) - qlTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) - table.storage.locationUri.foreach { loc => shim.setDataLocation(qlTable, loc) } - table.storage.inputFormat.map(toInputFormat).foreach(qlTable.setInputFormatClass) - table.storage.outputFormat.map(toOutputFormat).foreach(qlTable.setOutputFormatClass) - table.storage.serde.foreach(qlTable.setSerializationLib) - table.storage.serdeProperties.foreach { case (k, v) => qlTable.setSerdeParam(k, v) } - table.properties.foreach { case (k, v) => qlTable.setProperty(k, v) } - table.viewOriginalText.foreach { t => qlTable.setViewOriginalText(t) } - table.viewText.foreach { t => qlTable.setViewExpandedText(t) } - qlTable + hiveTable.setOwner(conf.getUser) + hiveTable.setNumBuckets(table.numBuckets) + hiveTable.setCreateTime((table.createTime / 1000).toInt) + hiveTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) + table.storage.locationUri.foreach { loc => shim.setDataLocation(hiveTable, loc) } + table.storage.inputFormat.map(toInputFormat).foreach(hiveTable.setInputFormatClass) + table.storage.outputFormat.map(toOutputFormat).foreach(hiveTable.setOutputFormatClass) + table.storage.serde.foreach(hiveTable.setSerializationLib) + table.storage.serdeProperties.foreach { case (k, v) => hiveTable.setSerdeParam(k, v) } + table.properties.foreach { case (k, v) => hiveTable.setProperty(k, v) } + table.viewOriginalText.foreach { t => hiveTable.setViewOriginalText(t) } + table.viewText.foreach { t => hiveTable.setViewExpandedText(t) } + hiveTable } private def toViewTable(view: CatalogTable): HiveTable = { - val tbl = toQlTable(view) + val tbl = toHiveTable(view) tbl.setTableType(HiveTableType.VIRTUAL_VIEW) tbl.setSerializationLib(null) tbl.clearSerDeInfo() @@ -382,7 +383,7 @@ private[hive] class HiveClientImpl( } override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState { - client.createTable(toQlTable(table), ignoreIfExists) + client.createTable(toHiveTable(table), ignoreIfExists) } override def dropTable( @@ -393,15 +394,19 @@ private[hive] class HiveClientImpl( } override def alterTable(tableName: String, table: CatalogTable): Unit = withHiveState { - val qlTable = toQlTable(table) + val hiveTable = toHiveTable(table) val qualifiedTableName = s"${table.database}.$tableName" - client.alterTable(qualifiedTableName, qlTable) + client.alterTable(qualifiedTableName, hiveTable) } - private def toCatalogPartition(partition: HivePartition): CatalogTablePartition = { - val apiPartition = partition.getTPartition + private def toHivePartition(p: CatalogTablePartition, ht: HiveTable): HivePartition = { + new HivePartition(ht, p.spec.asJava, p.storage.locationUri.map { l => new Path(l) }.orNull) + } + + private def fromHivePartition(hp: HivePartition): CatalogTablePartition = { + val apiPartition = hp.getTPartition CatalogTablePartition( - spec = Option(apiPartition.getValues).map(_.asScala).getOrElse(Seq.empty), + spec = Option(hp.getSpec).map(_.asScala.toMap).getOrElse(Map.empty), storage = CatalogStorageFormat( locationUri = Option(apiPartition.getSd.getLocation), inputFormat = Option(apiPartition.getSd.getInputFormat), @@ -410,24 +415,73 @@ private[hive] class HiveClientImpl( serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) } + override def createPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = withHiveState { + val addPartitionDesc = new AddPartitionDesc(db, table, ignoreIfExists) + parts.foreach { s => + addPartitionDesc.addPartition(s.spec.asJava, s.storage.locationUri.orNull) + } + client.createPartitions(addPartitionDesc) + } + + override def dropPartitions( + db: String, + table: String, + specs: Seq[Catalog.TablePartitionSpec], + ignoreIfNotExists: Boolean): Unit = withHiveState { + // TODO: figure out how to drop multiple partitions in one call + specs.foreach { s => + val dropOptions = new PartitionDropOptions + dropOptions.ifExists = ignoreIfNotExists + client.dropPartition(db, table, s.values.toList.asJava, dropOptions) + } + } + + override def renamePartitions( + db: String, + table: String, + specs: Seq[Catalog.TablePartitionSpec], + newSpecs: Seq[Catalog.TablePartitionSpec]): Unit = withHiveState { + assert(specs.size == newSpecs.size, "number of old and new partition specs differ") + val catalogTable = getTable(db, table) + val hiveTable = toHiveTable(catalogTable) + specs.zip(newSpecs).foreach { case (oldSpec, newSpec) => + val hivePart = getPartitionOption(catalogTable, oldSpec) + .map { p => toHivePartition(p.copy(spec = newSpec), hiveTable) } + .getOrElse { throw new NoSuchPartitionException } + client.renamePartition(hiveTable, oldSpec.asJava, hivePart) + } + } + + override def alterPartitions( + db: String, + table: String, + newParts: Seq[CatalogTablePartition]): Unit = withHiveState { + val hiveTable = toHiveTable(getTable(db, table)) + client.alterPartitions(table, newParts.map { p => toHivePartition(p, hiveTable) }.asJava) + } + override def getPartitionOption( table: CatalogTable, - partitionSpec: JMap[String, String]): Option[CatalogTablePartition] = withHiveState { - val qlTable = toQlTable(table) - val qlPartition = client.getPartition(qlTable, partitionSpec, false) - Option(qlPartition).map(toCatalogPartition) + spec: Catalog.TablePartitionSpec): Option[CatalogTablePartition] = withHiveState { + val hiveTable = toHiveTable(table) + val hivePartition = client.getPartition(hiveTable, spec.asJava, false) + Option(hivePartition).map(fromHivePartition) } override def getAllPartitions(table: CatalogTable): Seq[CatalogTablePartition] = withHiveState { - val qlTable = toQlTable(table) - shim.getAllPartitions(client, qlTable).map(toCatalogPartition) + val hiveTable = toHiveTable(table) + shim.getAllPartitions(client, hiveTable).map(fromHivePartition) } override def getPartitionsByFilter( table: CatalogTable, predicates: Seq[Expression]): Seq[CatalogTablePartition] = withHiveState { - val qlTable = toQlTable(table) - shim.getPartitionsByFilter(client, qlTable, predicates).map(toCatalogPartition) + val hiveTable = toHiveTable(table) + shim.getPartitionsByFilter(client, hiveTable, predicates).map(fromHivePartition) } override def listTables(dbName: String): Seq[String] = withHiveState { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index feb133d44898a..d316664241f1a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -205,7 +205,7 @@ case class InsertIntoHiveTable( val oldPart = catalog.client.getPartitionOption( catalog.client.getTable(table.databaseName, table.tableName), - partitionSpec.asJava) + partitionSpec) if (oldPart.isEmpty || !ifNotExists) { catalog.client.loadPartition( From 3ceb88d51a6e6af92cff2e90622ba235d0d107e9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 12 Feb 2016 14:04:45 -0800 Subject: [PATCH 15/30] Implement all function operations --- .../spark/sql/catalyst/analysis/Catalog.scala | 1 + .../catalyst/catalog/InMemoryCatalog.scala | 27 +++++------ .../sql/catalyst/catalog/interface.scala | 9 ++-- .../catalyst/catalog/CatalogTestCases.scala | 43 ++++++++++------- .../apache/spark/sql/hive/HiveCatalog.scala | 27 +++++------ .../spark/sql/hive/client/HiveClient.scala | 23 +++++++++ .../sql/hive/client/HiveClientImpl.scala | 48 ++++++++++++++++--- 7 files changed, 120 insertions(+), 58 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index c2f2581872b82..b825164bcf935 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} */ abstract class NoSuchItemException extends Exception +class NoSuchFunctionException extends NoSuchItemException class NoSuchPartitionException extends NoSuchItemException class NoSuchTableException extends NoSuchItemException class NoSuchDatabaseException extends NoSuchItemException 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 c513f0956913c..964eccfca4a2e 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 @@ -291,15 +291,10 @@ class InMemoryCatalog extends Catalog { // Functions // -------------------------------------------------------------------------- - override def createFunction( - db: String, - func: CatalogFunction, - ignoreIfExists: Boolean): Unit = synchronized { + override def createFunction(db: String, func: CatalogFunction): Unit = synchronized { assertDbExists(db) if (existsFunction(db, func.name)) { - if (!ignoreIfExists) { - 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, func) } @@ -310,15 +305,15 @@ class InMemoryCatalog extends Catalog { catalog(db).functions.remove(funcName) } - override def alterFunction( - db: String, - funcName: String, - funcDefinition: CatalogFunction): Unit = synchronized { - assertFunctionExists(db, funcName) - if (funcName != funcDefinition.name) { - // Also a rename; remove the old one and add the new one back - catalog(db).functions.remove(funcName) - } + override def renameFunction(db: String, oldName: String, newName: String): Unit = synchronized { + assertFunctionExists(db, oldName) + val newFunc = getFunction(db, oldName).copy(name = newName) + catalog(db).functions.remove(oldName) + catalog(db).functions.put(newName, newFunc) + } + + override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = synchronized { + assertFunctionExists(db, funcDefinition.name) catalog(db).functions.put(funcDefinition.name, funcDefinition) } 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 3ba627fe073f0..8867994568f8b 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 @@ -111,14 +111,13 @@ abstract class Catalog { // Functions // -------------------------------------------------------------------------- - def createFunction(db: String, funcDefinition: CatalogFunction, ignoreIfExists: Boolean): Unit + def createFunction(db: String, funcDefinition: CatalogFunction): Unit def dropFunction(db: String, funcName: String): Unit - /** - * Alter an existing function and optionally override its name. - */ - def alterFunction(db: String, funcName: String, funcDefinition: CatalogFunction): Unit + def renameFunction(db: String, oldName: String, newName: String): Unit + + def alterFunction(db: String, funcDefinition: CatalogFunction): Unit def getFunction(db: String, funcName: String): CatalogFunction 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 55f38630413f5..1c750faac0170 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 @@ -53,7 +53,7 @@ abstract class CatalogTestCases extends SparkFunSuite { catalog.createTable("db2", newTable("tbl1"), ignoreIfExists = false) catalog.createTable("db2", newTable("tbl2"), ignoreIfExists = false) catalog.createPartitions("db2", "tbl2", Seq(part1, part2), ignoreIfExists = false) - catalog.createFunction("db2", newFunc("func1"), ignoreIfExists = false) + catalog.createFunction("db2", newFunc("func1")) catalog } @@ -401,23 +401,22 @@ abstract class CatalogTestCases extends SparkFunSuite { test("basic create and list functions") { val catalog = newEmptyCatalog() catalog.createDatabase(newDb("mydb"), ignoreIfExists = false) - catalog.createFunction("mydb", newFunc("myfunc"), ignoreIfExists = false) + catalog.createFunction("mydb", newFunc("myfunc")) assert(catalog.listFunctions("mydb", "*").toSet == Set("myfunc")) } test("create function when database does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { - catalog.createFunction("does_not_exist", newFunc(), ignoreIfExists = false) + catalog.createFunction("does_not_exist", newFunc()) } } test("create function that already exists") { val catalog = newBasicCatalog() intercept[AnalysisException] { - catalog.createFunction("db2", newFunc("func1"), ignoreIfExists = false) + catalog.createFunction("db2", newFunc("func1")) } - catalog.createFunction("db2", newFunc("func1"), ignoreIfExists = true) } test("drop function") { @@ -456,31 +455,43 @@ abstract class CatalogTestCases extends SparkFunSuite { } } - test("alter function") { + test("rename function") { val catalog = newBasicCatalog() + val newName = "funcky" assert(catalog.getFunction("db2", "func1").className == funcClass) - // alter func but keep name - catalog.alterFunction("db2", "func1", newFunc("func1").copy(className = "muhaha")) - assert(catalog.getFunction("db2", "func1").className == "muhaha") - // alter func and change name - catalog.alterFunction("db2", "func1", newFunc("funcky")) + catalog.renameFunction("db2", "func1", newName) + intercept[AnalysisException] { catalog.getFunction("db2", "func1") } + assert(catalog.getFunction("db2", newName).name == newName) + assert(catalog.getFunction("db2", newName).className == funcClass) + intercept[AnalysisException] { catalog.renameFunction("db2", "does_not_exist", "me") } + } + + test("rename function when database does not exist") { + val catalog = newBasicCatalog() intercept[AnalysisException] { - catalog.getFunction("db2", "func1") + catalog.renameFunction("does_not_exist", "func1", "func5") } - assert(catalog.getFunction("db2", "funcky").className == funcClass) + } + + test("alter function") { + val catalog = newBasicCatalog() + assert(catalog.getFunction("db2", "func1").className == funcClass) + catalog.alterFunction("db2", newFunc("func1").copy(className = "muhaha")) + assert(catalog.getFunction("db2", "func1").className == "muhaha") + intercept[AnalysisException] { catalog.alterFunction("db2", newFunc("funcky")) } } test("alter function when database does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { - catalog.alterFunction("does_not_exist", "func1", newFunc()) + catalog.alterFunction("does_not_exist", newFunc()) } } test("list functions") { val catalog = newBasicCatalog() - catalog.createFunction("db2", newFunc("func2"), ignoreIfExists = false) - catalog.createFunction("db2", newFunc("not_me"), ignoreIfExists = false) + catalog.createFunction("db2", newFunc("func2")) + catalog.createFunction("db2", newFunc("not_me")) assert(catalog.listFunctions("db2", "*").toSet == Set("func1", "func2", "not_me")) assert(catalog.listFunctions("db2", "func*").toSet == Set("func1", "func2")) } 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 ca4b25e0be26c..b3370b609a5f5 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 @@ -173,31 +173,28 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def createFunction( db: String, - funcDefinition: CatalogFunction, - ignoreIfExists: Boolean): Unit = synchronized { - throw new UnsupportedOperationException + funcDefinition: CatalogFunction): Unit = synchronized { + client.createFunction(db, funcDefinition) } - override def dropFunction(db: String, funcName: String): Unit = synchronized { - throw new UnsupportedOperationException + override def dropFunction(db: String, name: String): Unit = synchronized { + client.dropFunction(db, name) } - /** - * Alter an existing function and optionally override its name. - */ - override def alterFunction( - db: String, - funcName: String, - funcDefinition: CatalogFunction): Unit = synchronized { - throw new UnsupportedOperationException + override def renameFunction(db: String, oldName: String, newName: String): Unit = synchronized { + client.renameFunction(db, oldName, newName) + } + + override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = synchronized { + client.alterFunction(db, funcDefinition) } override def getFunction(db: String, funcName: String): CatalogFunction = synchronized { - throw new UnsupportedOperationException + client.getFunction(db, funcName) } override def listFunctions(db: String, pattern: String): Seq[String] = synchronized { - throw new UnsupportedOperationException + client.listFunctions(db, pattern) } } 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 4564f15135918..b142e022a71a0 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 @@ -211,6 +211,29 @@ private[hive] trait HiveClient { holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit + /** Create a function in an existing database. */ + def createFunction(db: String, func: CatalogFunction): Unit + + /** Drop an existing function an the database. */ + def dropFunction(db: String, name: String): Unit + + /** Rename an existing function in the database. */ + def renameFunction(db: String, oldName: String, newName: String): Unit + + /** Alter an existing function in the database. */ + def alterFunction(db: String, func: CatalogFunction): Unit + + /** Return an existing function in the database, assuming it exists. */ + final def getFunction(db: String, name: String): CatalogFunction = { + getFunctionOption(db, name).getOrElse(throw new NoSuchFunctionException) + } + + /** Return an existing function in the database, or None if it doesn't exist. */ + def getFunctionOption(db: String, name: String): Option[CatalogFunction] + + /** Return the names of all functions that match the given pattern in the database. */ + def listFunctions(db: String, pattern: String): Seq[String] + /** Add a jar into class loader */ def addJar(path: String): Unit diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index d63ce4572e342..1dc472de4dded 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 @@ -25,7 +25,8 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{PartitionDropOptions, TableType => HiveTableType} -import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} +import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, + FieldSchema, Function => HiveFunction, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.AddPartitionDesc @@ -35,7 +36,6 @@ import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader} import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkConf, SparkException} -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression @@ -328,16 +328,24 @@ private[hive] class HiveClientImpl( Utils.classForName(name) .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] + private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { + new HiveFunction(f.name, db, f.className, null, null, -1, null, List.empty[ResourceUri].asJava) + } + + private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { + new CatalogFunction(hf.getFunctionName, hf.getClassName) + } + private def toHiveColumn(c: CatalogColumn): FieldSchema = { new FieldSchema(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment.orNull) } - private def fromHiveColumn(f: FieldSchema): CatalogColumn = { + private def fromHiveColumn(hc: FieldSchema): CatalogColumn = { new CatalogColumn( - name = f.getName, - dataType = HiveMetastoreTypes.toDataType(f.getType), + name = hc.getName, + dataType = HiveMetastoreTypes.toDataType(hc.getType), nullable = true, - comment = Option(f.getComment)) + comment = Option(hc.getComment)) } private def toHiveTable(table: CatalogTable): HiveTable = { @@ -605,6 +613,34 @@ private[hive] class HiveClientImpl( listBucketingEnabled) } + override def createFunction(db: String, func: CatalogFunction): Unit = withHiveState { + client.createFunction(toHiveFunction(func, db)) + } + + override def dropFunction(db: String, name: String): Unit = withHiveState { + client.dropFunction(db, name) + } + + override def renameFunction(db: String, oldName: String, newName: String): Unit = withHiveState { + val catalogFunc = getFunction(db, oldName).copy(name = newName) + 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)) + } + + override def getFunctionOption( + db: String, + name: String): Option[CatalogFunction] = withHiveState { + Option(client.getFunction(db, name)).map(fromHiveFunction) + } + + override def listFunctions(db: String, pattern: String): Seq[String] = withHiveState { + client.getFunctions(db, pattern).asScala + } + def addJar(path: String): Unit = { val uri = new Path(path).toUri val jarURL = if (uri.getScheme == null) { From 07332ad6803e578d9a61cc4693d8ce665ad8c29a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 12 Feb 2016 14:10:33 -0800 Subject: [PATCH 16/30] Simplify alterDatabase The operation doesn't support renaming anyway, so it doesn't make sense to pass in a name AND a CatalogDatabase that always has the same name. --- .../spark/sql/catalyst/catalog/InMemoryCatalog.scala | 7 +++---- .../org/apache/spark/sql/catalyst/catalog/interface.scala | 5 +---- .../spark/sql/catalyst/catalog/CatalogTestCases.scala | 4 ++-- .../scala/org/apache/spark/sql/hive/HiveCatalog.scala | 7 ++----- .../org/apache/spark/sql/hive/client/HiveClient.scala | 4 ++-- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 8 ++------ 6 files changed, 12 insertions(+), 23 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 964eccfca4a2e..8687a50f8fd2e 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 @@ -125,10 +125,9 @@ class InMemoryCatalog extends Catalog { } } - override def alterDatabase(db: String, dbDefinition: CatalogDatabase): Unit = synchronized { - assertDbExists(db) - assert(db == dbDefinition.name) - catalog(db).db = dbDefinition + override def alterDatabase(dbDefinition: CatalogDatabase): Unit = synchronized { + assertDbExists(dbDefinition.name) + catalog(dbDefinition.name).db = dbDefinition } override def getDatabase(db: String): CatalogDatabase = 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 8867994568f8b..ef6196eda80f6 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 @@ -40,10 +40,7 @@ abstract class Catalog { def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit - /** - * Alter an existing database. This operation does not support renaming. - */ - def alterDatabase(db: String, dbDefinition: CatalogDatabase): Unit + def alterDatabase(dbDefinition: CatalogDatabase): Unit def getDatabase(db: String): CatalogDatabase 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 1c750faac0170..a3729d5cd786f 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 @@ -154,13 +154,13 @@ abstract class CatalogTestCases extends SparkFunSuite { test("alter database") { val catalog = newBasicCatalog() - catalog.alterDatabase("db1", CatalogDatabase("db1", "new description", "lll", Map.empty)) + catalog.alterDatabase(CatalogDatabase("db1", "new description", "lll", Map.empty)) assert(catalog.getDatabase("db1").description == "new description") } test("alter database should throw exception when the database does not exist") { intercept[AnalysisException] { - newBasicCatalog().alterDatabase("no_db", CatalogDatabase("no_db", "ddd", "lll", Map.empty)) + newBasicCatalog().alterDatabase(CatalogDatabase("no_db", "ddd", "lll", Map.empty)) } } 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 b3370b609a5f5..01ec2eb6d1f64 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 @@ -54,11 +54,8 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { client.dropDatabase(db, ignoreIfNotExists, cascade) } - /** - * Alter an existing database. This operation does not support renaming. - */ - override def alterDatabase(db: String, dbDefinition: CatalogDatabase): Unit = synchronized { - client.alterDatabase(db, dbDefinition) + override def alterDatabase(dbDefinition: CatalogDatabase): Unit = synchronized { + client.alterDatabase(dbDefinition) } override def getDatabase(db: String): CatalogDatabase = synchronized { 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 b142e022a71a0..738dad7e74e80 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 @@ -111,9 +111,9 @@ private[hive] trait HiveClient { def dropDatabase(name: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit /** - * Alter an existing database. This operation does not support renaming. + * Alter an existing database. */ - def alterDatabase(name: String, database: CatalogDatabase): Unit + def alterDatabase(database: CatalogDatabase): Unit /** * Create one or many partitions in the given table. 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 1dc472de4dded..2b2f1fc3be061 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 @@ -260,13 +260,9 @@ private[hive] class HiveClientImpl( client.dropDatabase(name, true, ignoreIfNotExists, cascade) } - /** - * Alter an existing database. This operation does not support renaming. - */ - override def alterDatabase(name: String, database: CatalogDatabase): Unit = withHiveState { - assert(name == database.name) + override def alterDatabase(database: CatalogDatabase): Unit = withHiveState { client.alterDatabase( - name, + database.name, new HiveDatabase( database.name, database.description, From cdf1f70479a6ac588249cea221b602e07d936892 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 12 Feb 2016 14:15:55 -0800 Subject: [PATCH 17/30] Clean up HiveClientImpl a little --- .../sql/hive/client/HiveClientImpl.scala | 164 +++++++++--------- 1 file changed, 85 insertions(+), 79 deletions(-) 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 2b2f1fc3be061..84011ab25ca13 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 @@ -317,73 +317,12 @@ private[hive] class HiveClientImpl( } } - private def toInputFormat(name: String) = - Utils.classForName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]] - - private def toOutputFormat(name: String) = - Utils.classForName(name) - .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] - - private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { - new HiveFunction(f.name, db, f.className, null, null, -1, null, List.empty[ResourceUri].asJava) - } - - private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { - new CatalogFunction(hf.getFunctionName, hf.getClassName) - } - - private def toHiveColumn(c: CatalogColumn): FieldSchema = { - new FieldSchema(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment.orNull) - } - - private def fromHiveColumn(hc: FieldSchema): CatalogColumn = { - new CatalogColumn( - name = hc.getName, - dataType = HiveMetastoreTypes.toDataType(hc.getType), - nullable = true, - comment = Option(hc.getComment)) - } - - private def toHiveTable(table: CatalogTable): HiveTable = { - val hiveTable = new HiveTable(table.database, table.name) - hiveTable.setTableType(table.tableType match { - case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE - case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE - case CatalogTableType.INDEX_TABLE => HiveTableType.INDEX_TABLE - case CatalogTableType.VIRTUAL_VIEW => HiveTableType.VIRTUAL_VIEW - }) - hiveTable.setFields(table.schema.map(toHiveColumn).asJava) - hiveTable.setPartCols(table.partitionColumns.map(toHiveColumn).asJava) - // TODO: set sort columns here too - hiveTable.setOwner(conf.getUser) - hiveTable.setNumBuckets(table.numBuckets) - hiveTable.setCreateTime((table.createTime / 1000).toInt) - hiveTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) - table.storage.locationUri.foreach { loc => shim.setDataLocation(hiveTable, loc) } - table.storage.inputFormat.map(toInputFormat).foreach(hiveTable.setInputFormatClass) - table.storage.outputFormat.map(toOutputFormat).foreach(hiveTable.setOutputFormatClass) - table.storage.serde.foreach(hiveTable.setSerializationLib) - table.storage.serdeProperties.foreach { case (k, v) => hiveTable.setSerdeParam(k, v) } - table.properties.foreach { case (k, v) => hiveTable.setProperty(k, v) } - table.viewOriginalText.foreach { t => hiveTable.setViewOriginalText(t) } - table.viewText.foreach { t => hiveTable.setViewExpandedText(t) } - hiveTable - } - - private def toViewTable(view: CatalogTable): HiveTable = { - val tbl = toHiveTable(view) - tbl.setTableType(HiveTableType.VIRTUAL_VIEW) - tbl.setSerializationLib(null) - tbl.clearSerDeInfo() - tbl - } - override def createView(view: CatalogTable): Unit = withHiveState { - client.createTable(toViewTable(view)) + client.createTable(toHiveViewTable(view)) } override def alertView(view: CatalogTable): Unit = withHiveState { - client.alterTable(view.qualifiedName, toViewTable(view)) + client.alterTable(view.qualifiedName, toHiveViewTable(view)) } override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState { @@ -403,22 +342,6 @@ private[hive] class HiveClientImpl( client.alterTable(qualifiedTableName, hiveTable) } - private def toHivePartition(p: CatalogTablePartition, ht: HiveTable): HivePartition = { - new HivePartition(ht, p.spec.asJava, p.storage.locationUri.map { l => new Path(l) }.orNull) - } - - private def fromHivePartition(hp: HivePartition): CatalogTablePartition = { - val apiPartition = hp.getTPartition - CatalogTablePartition( - spec = Option(hp.getSpec).map(_.asScala.toMap).getOrElse(Map.empty), - storage = CatalogStorageFormat( - locationUri = Option(apiPartition.getSd.getLocation), - inputFormat = Option(apiPartition.getSd.getInputFormat), - outputFormat = Option(apiPartition.getSd.getOutputFormat), - serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), - serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) - } - override def createPartitions( db: String, table: String, @@ -670,4 +593,87 @@ private[hive] class HiveClientImpl( client.dropDatabase(db, true, false, true) } } + + + /* -------------------------------------------------------- * + | Helper methods for converting to and from Hive classes | + * -------------------------------------------------------- */ + + private def toInputFormat(name: String) = + Utils.classForName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]] + + private def toOutputFormat(name: String) = + Utils.classForName(name) + .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] + + private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { + new HiveFunction(f.name, db, f.className, null, null, -1, null, List.empty[ResourceUri].asJava) + } + + private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { + new CatalogFunction(hf.getFunctionName, hf.getClassName) + } + + private def toHiveColumn(c: CatalogColumn): FieldSchema = { + new FieldSchema(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment.orNull) + } + + private def fromHiveColumn(hc: FieldSchema): CatalogColumn = { + new CatalogColumn( + name = hc.getName, + dataType = HiveMetastoreTypes.toDataType(hc.getType), + nullable = true, + comment = Option(hc.getComment)) + } + + private def toHiveTable(table: CatalogTable): HiveTable = { + val hiveTable = new HiveTable(table.database, table.name) + hiveTable.setTableType(table.tableType match { + case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE + case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE + case CatalogTableType.INDEX_TABLE => HiveTableType.INDEX_TABLE + case CatalogTableType.VIRTUAL_VIEW => HiveTableType.VIRTUAL_VIEW + }) + hiveTable.setFields(table.schema.map(toHiveColumn).asJava) + hiveTable.setPartCols(table.partitionColumns.map(toHiveColumn).asJava) + // TODO: set sort columns here too + hiveTable.setOwner(conf.getUser) + hiveTable.setNumBuckets(table.numBuckets) + hiveTable.setCreateTime((table.createTime / 1000).toInt) + hiveTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) + table.storage.locationUri.foreach { loc => shim.setDataLocation(hiveTable, loc) } + table.storage.inputFormat.map(toInputFormat).foreach(hiveTable.setInputFormatClass) + table.storage.outputFormat.map(toOutputFormat).foreach(hiveTable.setOutputFormatClass) + table.storage.serde.foreach(hiveTable.setSerializationLib) + table.storage.serdeProperties.foreach { case (k, v) => hiveTable.setSerdeParam(k, v) } + table.properties.foreach { case (k, v) => hiveTable.setProperty(k, v) } + table.viewOriginalText.foreach { t => hiveTable.setViewOriginalText(t) } + table.viewText.foreach { t => hiveTable.setViewExpandedText(t) } + hiveTable + } + + private def toHiveViewTable(view: CatalogTable): HiveTable = { + val tbl = toHiveTable(view) + tbl.setTableType(HiveTableType.VIRTUAL_VIEW) + tbl.setSerializationLib(null) + tbl.clearSerDeInfo() + tbl + } + + private def toHivePartition(p: CatalogTablePartition, ht: HiveTable): HivePartition = { + new HivePartition(ht, p.spec.asJava, p.storage.locationUri.map { l => new Path(l) }.orNull) + } + + private def fromHivePartition(hp: HivePartition): CatalogTablePartition = { + val apiPartition = hp.getTPartition + CatalogTablePartition( + spec = Option(hp.getSpec).map(_.asScala.toMap).getOrElse(Map.empty), + storage = CatalogStorageFormat( + locationUri = Option(apiPartition.getSd.getLocation), + inputFormat = Option(apiPartition.getSd.getInputFormat), + outputFormat = Option(apiPartition.getSd.getOutputFormat), + serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), + serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) + } + } From 2b720256a319c9f9709801cb690f61cf1dbd0ace Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 12 Feb 2016 15:13:01 -0800 Subject: [PATCH 18/30] Fix tests? --- .../scala/org/apache/spark/sql/catalyst/catalog/interface.scala | 2 +- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 ef6196eda80f6..9c355a33d8918 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 @@ -197,7 +197,7 @@ case class CatalogTable( } -class CatalogTableType private(name: String) +case class CatalogTableType private(name: String) object CatalogTableType { val EXTERNAL_TABLE = new CatalogTableType("EXTERNAL_TABLE") val MANAGED_TABLE = new CatalogTableType("MANAGED_TABLE") 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 80d7e40e338dd..702c335033ca7 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 @@ -748,7 +748,7 @@ private[hive] case class MetastoreRelation( databaseName: String, tableName: String, alias: Option[String]) - (val table: CatalogTable, client: HiveClient) + (val table: CatalogTable, @transient private val client: HiveClient) (@transient private val sqlContext: SQLContext) extends LeafNode with MultiInstanceRelation with FileRelation { From 5e2cd3afe77333ee586cb0fdfe962856b1ba2e84 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 12 Feb 2016 15:54:32 -0800 Subject: [PATCH 19/30] Miscellaneous cleanup --- .../catalyst/catalog/InMemoryCatalog.scala | 2 +- .../sql/catalyst/catalog/interface.scala | 12 ++ .../catalyst/catalog/CatalogTestCases.scala | 4 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 14 +-- .../org/apache/spark/sql/hive/HiveQl.scala | 119 +++++++----------- .../hive/execution/CreateTableAsSelect.scala | 18 ++- 6 files changed, 79 insertions(+), 90 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 8687a50f8fd2e..15acb78d32048 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 @@ -39,7 +39,7 @@ class InMemoryCatalog extends Catalog { val functions = new mutable.HashMap[String, CatalogFunction] } - // Database name -> spec + // Database name -> description private val catalog = new scala.collection.mutable.HashMap[String, DatabaseDesc] private def filterPattern(names: Seq[String], pattern: String): Seq[String] = { 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 9c355a33d8918..a279bd1342a79 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 @@ -194,6 +194,18 @@ case class CatalogTable( /** Return the fully qualified name of this table, assuming the database was specified. */ def qualifiedName: String = s"$database.$name" + + /** Syntactic sugar to update a field in `storage`. */ + def withNewStorage( + locationUri: Option[String] = storage.locationUri, + inputFormat: Option[String] = storage.inputFormat, + outputFormat: Option[String] = storage.outputFormat, + serde: Option[String] = storage.serde, + serdeProperties: Map[String, String] = storage.serdeProperties): CatalogTable = { + copy(storage = CatalogStorageFormat( + locationUri, inputFormat, outputFormat, serde, serdeProperties)) + } + } 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 a3729d5cd786f..4efdc5781137f 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 @@ -86,9 +86,11 @@ abstract class CatalogTestCases extends SparkFunSuite { val catalog = newEmptyCatalog() catalog.createDatabase(newDb(), ignoreIfExists = false) assert(catalog.listDatabases().toSet == Set("default")) - catalog.createDatabase(newDb("default2"), ignoreIfExists = false) assert(catalog.listDatabases().toSet == Set("default", "default2")) + assert(catalog.databaseExists("default")) + assert(catalog.databaseExists("default2")) + assert(!catalog.databaseExists("does_not_exist")) } test("get database when a database exists") { 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 702c335033ca7..d8902a66c59c9 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 @@ -432,7 +432,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } } else { MetastoreRelation( - qualifiedTableName.database, qualifiedTableName.name, alias)(table, client)(hive) + qualifiedTableName.database, qualifiedTableName.name, alias)(table, client, hive) } } @@ -633,9 +633,8 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } else { val desc = if (table.storage.serde.isEmpty) { // add default serde - table.copy(storage = - table.storage.copy(serde = - Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"))) + table.withNewStorage( + serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) } else { table } @@ -748,8 +747,9 @@ private[hive] case class MetastoreRelation( databaseName: String, tableName: String, alias: Option[String]) - (val table: CatalogTable, @transient private val client: HiveClient) - (@transient private val sqlContext: SQLContext) + (val table: CatalogTable, + @transient private val client: HiveClient, + @transient private val sqlContext: SQLContext) extends LeafNode with MultiInstanceRelation with FileRelation { override def equals(other: Any): Boolean = other match { @@ -920,7 +920,7 @@ private[hive] case class MetastoreRelation( override def newInstance(): MetastoreRelation = { - MetastoreRelation(databaseName, tableName, alias)(table, client)(sqlContext) + MetastoreRelation(databaseName, tableName, alias)(table, client, sqlContext) } } 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 dbe02e02b89ea..6aee78e0a5962 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 @@ -399,11 +399,10 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) } - var storDesc = tableDesc.storage - hiveSerDe.inputFormat.foreach { i => storDesc = storDesc.copy(inputFormat = Some(i)) } - hiveSerDe.outputFormat.foreach { o => storDesc = storDesc.copy(outputFormat = Some(o)) } - hiveSerDe.serde.foreach { s => storDesc = storDesc.copy(serde = Some(s)) } - tableDesc = tableDesc.copy(storage = storDesc) + tableDesc = tableDesc.withNewStorage( + inputFormat = hiveSerDe.inputFormat.orElse(tableDesc.storage.inputFormat), + outputFormat = hiveSerDe.inputFormat.orElse(tableDesc.storage.outputFormat), + serde = hiveSerDe.inputFormat.orElse(tableDesc.storage.serde)) children.collect { case list @ Token("TOK_TABCOLLIST", _) => @@ -449,17 +448,14 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging // TODO support the nullFormat case _ => assert(false) } - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams.asScala)) + tableDesc = tableDesc.withNewStorage( + serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams.asScala) case Token("TOK_TABLELOCATION", child :: Nil) => val location = EximUtil.relativeToAbsolutePath(hiveConf, unescapeSQLString(child.text)) - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy(locationUri = Option(location))) + tableDesc = tableDesc.withNewStorage(locationUri = Option(location)) case Token("TOK_TABLESERIALIZER", child :: Nil) => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serde = Option(unescapeSQLString(child.children.head.text)))) + tableDesc = tableDesc.withNewStorage( + serde = Option(unescapeSQLString(child.children.head.text))) if (child.numChildren == 2) { // This is based on the readProps(..) method in // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java: @@ -471,74 +467,60 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging .orNull (unescapeSQLString(prop), value) }.toMap - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams)) + tableDesc = tableDesc.withNewStorage( + serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams) } case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) => child.text.toLowerCase(Locale.ENGLISH) match { case "orc" => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"))) + tableDesc = tableDesc.withNewStorage( + inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) if (tableDesc.storage.serde.isEmpty) { - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde"))) + tableDesc = tableDesc.withNewStorage( + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) } case "parquet" => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))) + tableDesc = tableDesc.withNewStorage( + inputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) if (tableDesc.storage.serde.isEmpty) { - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serde = - Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))) + tableDesc = tableDesc.withNewStorage( + serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) } case "rcfile" => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))) + tableDesc = tableDesc.withNewStorage( + inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) if (tableDesc.storage.serde.isEmpty) { - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serde = - Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))) + tableDesc = tableDesc.withNewStorage( + serde = + Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) } case "textfile" => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = - Option("org.apache.hadoop.mapred.TextInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))) + tableDesc = tableDesc.withNewStorage( + inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) case "sequencefile" => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), - outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat"))) + tableDesc = tableDesc.withNewStorage( + inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), + outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")) case "avro" => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"))) + tableDesc = tableDesc.withNewStorage( + inputFormat = + Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat")) if (tableDesc.storage.serde.isEmpty) { - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))) + tableDesc = tableDesc.withNewStorage( + serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) } case _ => @@ -548,26 +530,21 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging case Token("TOK_TABLESERIALIZER", Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy(serde = Option(unquoteString(serdeName)))) + tableDesc = tableDesc.withNewStorage(serde = Option(unquoteString(serdeName))) otherProps match { case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - serdeProperties = tableDesc.storage.serdeProperties ++ getProperties(list))) + tableDesc = tableDesc.withNewStorage( + serdeProperties = tableDesc.storage.serdeProperties ++ getProperties(list)) case _ => } case Token("TOK_TABLEPROPERTIES", list :: Nil) => tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) case list @ Token("TOK_TABLEFILEFORMAT", _) => - tableDesc = tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = - Option(unescapeSQLString(list.children.head.text)), - outputFormat = - Option(unescapeSQLString(list.children(1).text)))) + tableDesc = tableDesc.withNewStorage( + inputFormat = Option(unescapeSQLString(list.children.head.text)), + outputFormat = Option(unescapeSQLString(list.children(1).text))) case Token("TOK_STORAGEHANDLER", _) => throw new AnalysisException( "CREATE TABLE AS SELECT cannot be used for a non-native 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 ce975bb44fbc1..2aa5ed8cd891f 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 @@ -51,20 +51,18 @@ case class CreateTableAsSelect( import org.apache.hadoop.mapred.TextInputFormat val withFormat = - tableDesc.copy( - storage = tableDesc.storage.copy( - inputFormat = - tableDesc.storage.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), - outputFormat = - tableDesc.storage.outputFormat - .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), - serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName())))) + tableDesc.withNewStorage( + inputFormat = + tableDesc.storage.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), + outputFormat = + tableDesc.storage.outputFormat + .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), + serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName))) val withSchema = if (withFormat.schema.isEmpty) { // Hive doesn't support specifying the column list for target table in CTAS // However we don't think SparkSQL should follow that. - tableDesc.copy( - schema = query.output.map { c => CatalogColumn(c.name, c.dataType) }) + tableDesc.copy(schema = query.output.map { c => CatalogColumn(c.name, c.dataType) }) } else { withFormat } From 7d58fac540694f21279f221b4fae489c6b4d1933 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Feb 2016 14:17:15 -0800 Subject: [PATCH 20/30] Address comments + minor cleanups --- .../catalyst/catalog/InMemoryCatalog.scala | 56 +++++++++---------- .../sql/catalyst/catalog/interface.scala | 16 ++++++ .../apache/spark/sql/hive/HiveCatalog.scala | 9 ++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/client/HiveClient.scala | 13 ++--- .../sql/hive/client/HiveClientImpl.scala | 3 +- 6 files changed, 57 insertions(+), 42 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 15acb78d32048..57617d835ae92 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 @@ -48,39 +48,39 @@ class InMemoryCatalog extends Catalog { } private def existsFunction(db: String, funcName: String): Boolean = { - assertDbExists(db) + requireDbExists(db) catalog(db).functions.contains(funcName) } private def existsTable(db: String, table: String): Boolean = { - assertDbExists(db) + requireDbExists(db) catalog(db).tables.contains(table) } private def existsPartition(db: String, table: String, spec: TablePartitionSpec): Boolean = { - assertTableExists(db, table) + requireTableExists(db, table) catalog(db).tables(table).partitions.contains(spec) } - private def assertDbExists(db: String): Unit = { + private def requireDbExists(db: String): Unit = { if (!catalog.contains(db)) { throw new AnalysisException(s"Database $db does not exist") } } - private def assertFunctionExists(db: String, funcName: String): Unit = { + private def requireFunctionExists(db: String, funcName: String): Unit = { if (!existsFunction(db, funcName)) { throw new AnalysisException(s"Function $funcName does not exist in $db database") } } - private def assertTableExists(db: String, table: String): Unit = { + private def requireTableExists(db: String, table: String): Unit = { if (!existsTable(db, table)) { throw new AnalysisException(s"Table $table does not exist in $db database") } } - private def assertPartitionExists(db: String, table: String, spec: TablePartitionSpec): Unit = { + 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") } @@ -126,12 +126,12 @@ class InMemoryCatalog extends Catalog { } override def alterDatabase(dbDefinition: CatalogDatabase): Unit = synchronized { - assertDbExists(dbDefinition.name) + requireDbExists(dbDefinition.name) catalog(dbDefinition.name).db = dbDefinition } override def getDatabase(db: String): CatalogDatabase = synchronized { - assertDbExists(db) + requireDbExists(db) catalog(db).db } @@ -155,7 +155,7 @@ class InMemoryCatalog extends Catalog { db: String, tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = synchronized { - assertDbExists(db) + requireDbExists(db) if (existsTable(db, tableDefinition.name)) { if (!ignoreIfExists) { throw new AnalysisException(s"Table ${tableDefinition.name} already exists in $db database") @@ -169,7 +169,7 @@ class InMemoryCatalog extends Catalog { db: String, table: String, ignoreIfNotExists: Boolean): Unit = synchronized { - assertDbExists(db) + requireDbExists(db) if (existsTable(db, table)) { catalog(db).tables.remove(table) } else { @@ -180,7 +180,7 @@ class InMemoryCatalog extends Catalog { } override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { - assertTableExists(db, oldName) + requireTableExists(db, oldName) val oldDesc = catalog(db).tables(oldName) oldDesc.table = oldDesc.table.copy(name = newName) catalog(db).tables.put(newName, oldDesc) @@ -188,22 +188,22 @@ class InMemoryCatalog extends Catalog { } override def alterTable(db: String, tableDefinition: CatalogTable): Unit = synchronized { - assertTableExists(db, tableDefinition.name) + requireTableExists(db, tableDefinition.name) catalog(db).tables(tableDefinition.name).table = tableDefinition } override def getTable(db: String, table: String): CatalogTable = synchronized { - assertTableExists(db, table) + requireTableExists(db, table) catalog(db).tables(table).table } override def listTables(db: String): Seq[String] = synchronized { - assertDbExists(db) + requireDbExists(db) catalog(db).tables.keySet.toSeq } override def listTables(db: String, pattern: String): Seq[String] = synchronized { - assertDbExists(db) + requireDbExists(db) filterPattern(listTables(db), pattern) } @@ -216,7 +216,7 @@ class InMemoryCatalog extends Catalog { table: String, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = synchronized { - assertTableExists(db, table) + requireTableExists(db, table) val existingParts = catalog(db).tables(table).partitions if (!ignoreIfExists) { val dupSpecs = parts.collect { case p if existingParts.contains(p.spec) => p.spec } @@ -234,7 +234,7 @@ class InMemoryCatalog extends Catalog { table: String, partSpecs: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = synchronized { - assertTableExists(db, table) + requireTableExists(db, table) val existingParts = catalog(db).tables(table).partitions if (!ignoreIfNotExists) { val missingSpecs = partSpecs.collect { case s if !existingParts.contains(s) => s } @@ -252,7 +252,7 @@ class InMemoryCatalog extends Catalog { table: String, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = synchronized { - assert(specs.size == newSpecs.size, "number of old and new partition specs differ") + require(specs.size == newSpecs.size, "number of old and new partition specs differ") specs.zip(newSpecs).foreach { case (oldSpec, newSpec) => val newPart = getPartition(db, table, oldSpec).copy(spec = newSpec) val existingParts = catalog(db).tables(table).partitions @@ -266,7 +266,7 @@ class InMemoryCatalog extends Catalog { table: String, parts: Seq[CatalogTablePartition]): Unit = synchronized { parts.foreach { p => - assertPartitionExists(db, table, p.spec) + requirePartitionExists(db, table, p.spec) catalog(db).tables(table).partitions.put(p.spec, p) } } @@ -275,14 +275,14 @@ class InMemoryCatalog extends Catalog { db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition = synchronized { - assertPartitionExists(db, table, spec) + requirePartitionExists(db, table, spec) catalog(db).tables(table).partitions(spec) } override def listPartitions( db: String, table: String): Seq[CatalogTablePartition] = synchronized { - assertTableExists(db, table) + requireTableExists(db, table) catalog(db).tables(table).partitions.values.toSeq } @@ -291,7 +291,7 @@ class InMemoryCatalog extends Catalog { // -------------------------------------------------------------------------- override def createFunction(db: String, func: CatalogFunction): Unit = synchronized { - assertDbExists(db) + requireDbExists(db) if (existsFunction(db, func.name)) { throw new AnalysisException(s"Function $func already exists in $db database") } else { @@ -300,29 +300,29 @@ class InMemoryCatalog extends Catalog { } override def dropFunction(db: String, funcName: String): Unit = synchronized { - assertFunctionExists(db, funcName) + requireFunctionExists(db, funcName) catalog(db).functions.remove(funcName) } override def renameFunction(db: String, oldName: String, newName: String): Unit = synchronized { - assertFunctionExists(db, oldName) + requireFunctionExists(db, oldName) val newFunc = getFunction(db, oldName).copy(name = newName) catalog(db).functions.remove(oldName) catalog(db).functions.put(newName, newFunc) } override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = synchronized { - assertFunctionExists(db, funcDefinition.name) + requireFunctionExists(db, funcDefinition.name) catalog(db).functions.put(funcDefinition.name, funcDefinition) } override def getFunction(db: String, funcName: String): CatalogFunction = synchronized { - assertFunctionExists(db, funcName) + requireFunctionExists(db, funcName) catalog(db).functions(funcName) } override def listFunctions(db: String, pattern: String): Seq[String] = synchronized { - assertDbExists(db) + requireDbExists(db) filterPattern(catalog(db).functions.keysIterator.toSeq, pattern) } 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 a279bd1342a79..42b18c484a632 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 @@ -40,6 +40,10 @@ abstract class Catalog { def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit + /** + * Alter a database whose name matches the one specified in `dbDefinition`, + * assuming the database exists. + */ def alterDatabase(dbDefinition: CatalogDatabase): Unit def getDatabase(db: String): CatalogDatabase @@ -60,6 +64,10 @@ abstract class Catalog { def renameTable(db: String, oldName: String, newName: String): Unit + /** + * Alter a table whose name that matches the one specified in `tableDefinition`, + * assuming the table exists. + */ def alterTable(db: String, tableDefinition: CatalogTable): Unit def getTable(db: String, table: String): CatalogTable @@ -94,6 +102,10 @@ abstract class Catalog { 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. + */ def alterPartitions( db: String, table: String, @@ -114,6 +126,10 @@ abstract class Catalog { def renameFunction(db: String, oldName: String, newName: String): Unit + /** + * Alter a function whose name that matches the one specified in `funcDefinition`, + * assuming the function exists. + */ def alterFunction(db: String, funcDefinition: CatalogFunction): Unit def getFunction(db: String, funcName: String): CatalogFunction 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 01ec2eb6d1f64..c5044ccb764eb 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 @@ -32,8 +32,8 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { /** * Assert that the provided database matches the one specified in the table. */ - private def assertDbMatches(db: String, table: CatalogTable): Unit = { - assert(table.specifiedDatabase == Some(db), + private def requireDbMatches(db: String, table: CatalogTable): Unit = { + require(table.specifiedDatabase == Some(db), "provided database does not much the one specified in the table definition") } @@ -82,7 +82,7 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = synchronized { - assertDbMatches(db, tableDefinition) + requireDbMatches(db, tableDefinition) client.createTable(tableDefinition, ignoreIfExists) } @@ -95,12 +95,11 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { val newTable = client.getTable(db, oldName).copy(name = newName) - assertDbMatches(db, newTable) client.alterTable(oldName, newTable) } override def alterTable(db: String, tableDefinition: CatalogTable): Unit = synchronized { - assertDbMatches(db, tableDefinition) + requireDbMatches(db, tableDefinition) 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 d8902a66c59c9..55afe49be90f5 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 @@ -311,7 +311,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte assert(partitionColumns.isEmpty) assert(relation.partitionColumns.isEmpty) - // TODO We need to place the SQL string in the view text here. + // TODO: We need to place the SQL string in the view text here. CatalogTable( specifiedDatabase = Option(dbName), name = tblName, 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 738dad7e74e80..4b789d8a31842 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 @@ -90,10 +90,8 @@ private[hive] trait HiveClient { /** Drop the specified table. */ def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean): Unit - /** Updates the given table with new metadata. */ - final def alterTable(table: CatalogTable): Unit = { - alterTable(table.qualifiedName, table) - } + /** Alter a table whose name matches the one specified in `table`, assuming it exists. */ + final def alterTable(table: CatalogTable): Unit = alterTable(table.name, table) /** Updates the given table with new metadata, optionally renaming the table. */ def alterTable(tableName: String, table: CatalogTable): Unit @@ -111,7 +109,7 @@ private[hive] trait HiveClient { def dropDatabase(name: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit /** - * Alter an existing database. + * Alter a database whose name matches the one specified in `database`, assuming it exists. */ def alterDatabase(database: CatalogDatabase): Unit @@ -143,7 +141,8 @@ private[hive] trait HiveClient { newSpecs: Seq[Catalog.TablePartitionSpec]): Unit /** - * Alter one or many existing table partitions, assuming they exist. + * Alter one or more table partitions whose specs match the ones specified in `newParts`, + * assuming the partitions exist. */ def alterPartitions( db: String, @@ -220,7 +219,7 @@ private[hive] trait HiveClient { /** Rename an existing function in the database. */ def renameFunction(db: String, oldName: String, newName: String): Unit - /** Alter an existing function in the database. */ + /** Alter a function whose name matches the one specified in `func`, assuming it exists. */ def alterFunction(db: String, func: CatalogFunction): Unit /** Return an existing function in the database, assuming it exists. */ 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 84011ab25ca13..1baa03870a5b2 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 @@ -338,6 +338,7 @@ private[hive] class HiveClientImpl( override def alterTable(tableName: String, table: CatalogTable): Unit = withHiveState { val hiveTable = toHiveTable(table) + // Do not use `table.qualifiedName` here because this may be a rename val qualifiedTableName = s"${table.database}.$tableName" client.alterTable(qualifiedTableName, hiveTable) } @@ -372,7 +373,7 @@ private[hive] class HiveClientImpl( table: String, specs: Seq[Catalog.TablePartitionSpec], newSpecs: Seq[Catalog.TablePartitionSpec]): Unit = withHiveState { - assert(specs.size == newSpecs.size, "number of old and new partition specs differ") + require(specs.size == newSpecs.size, "number of old and new partition specs differ") val catalogTable = getTable(db, table) val hiveTable = toHiveTable(catalogTable) specs.zip(newSpecs).foreach { case (oldSpec, newSpec) => From 1c05b9b3ce677a62062f1d90f861b20398ab42a4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Feb 2016 14:33:13 -0800 Subject: [PATCH 21/30] Fix wrong Hive TableType issue We used to pass CatalogTableType#toString into HiveTable, which fails later when Hive extracts the Java enum value from the string. This was the cause of test failures in a few test suites: - InsertIntoHiveTableSuite - MultiDatabaseSuite - ParquetMetastoreSuite - ... --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 55afe49be90f5..0d73cb494f0b8 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 @@ -25,7 +25,7 @@ import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.Warehouse +import org.apache.hadoop.hive.metastore.{TableType => HiveTableType, Warehouse} import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata.{Table => HiveTable, _} import org.apache.hadoop.hive.ql.plan.TableDesc @@ -782,7 +782,12 @@ private[hive] case class MetastoreRelation( tTable.setParameters(tableParameters) table.properties.foreach { case (k, v) => tableParameters.put(k, v) } - tTable.setTableType(table.tableType.toString) + tTable.setTableType(table.tableType match { + case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE.toString + case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE.toString + case CatalogTableType.INDEX_TABLE => HiveTableType.INDEX_TABLE.toString + case CatalogTableType.VIRTUAL_VIEW => HiveTableType.VIRTUAL_VIEW.toString + }) val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tTable.setSd(sd) From 4ecc3b1245998d2c9743840d1243ec55770db1a9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Feb 2016 14:55:25 -0800 Subject: [PATCH 22/30] Fix CREATE TABLE serde setting Blatant programming mistake. This was caught by hive.execution.SQLQuerySuite. --- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 6aee78e0a5962..43131e84b3693 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 @@ -401,8 +401,8 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging tableDesc = tableDesc.withNewStorage( inputFormat = hiveSerDe.inputFormat.orElse(tableDesc.storage.inputFormat), - outputFormat = hiveSerDe.inputFormat.orElse(tableDesc.storage.outputFormat), - serde = hiveSerDe.inputFormat.orElse(tableDesc.storage.serde)) + outputFormat = hiveSerDe.outputFormat.orElse(tableDesc.storage.outputFormat), + serde = hiveSerDe.serde.orElse(tableDesc.storage.serde)) children.collect { case list @ Token("TOK_TABCOLLIST", _) => From 863ebd095e7c36c740ad88ec671522a4550f0273 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Feb 2016 15:22:54 -0800 Subject: [PATCH 23/30] Fix NPE in CREATE VIEW When we create views using HiveQl we pass in null data types because we can't specify these types until later. This caused a NPE downstream. --- .../scala/org/apache/spark/sql/catalyst/CatalystQl.scala | 1 + .../org/apache/spark/sql/catalyst/catalog/interface.scala | 4 +++- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 8 +++++--- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 8099751900a42..1a50e8d3ede20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -52,6 +52,7 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends case e: MatchError => throw e case e: AnalysisException => throw e case e: Exception => + // TODO: don't swallow original stack trace throw new AnalysisException(e.getMessage) case e: NotImplementedError => throw new AnalysisException( 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 42b18c484a632..99e0b7446ecbd 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.catalog +import javax.annotation.Nullable + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.types.DataType @@ -164,7 +166,7 @@ case class CatalogStorageFormat( */ case class CatalogColumn( name: String, - dataType: DataType, + @Nullable dataType: DataType, // may be null when used to create views nullable: Boolean = true, comment: Option[String] = None) 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 43131e84b3693..a4c3ef8473b8a 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 @@ -687,11 +687,13 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging protected def nodeToColumns(node: ASTNode, lowerCase: Boolean): Seq[CatalogColumn] = { node.children.map(_.children).collect { case Token(rawColName, Nil) :: colTypeNode :: comment => - val colName = if (!lowerCase) rawColName - else rawColName.toLowerCase + val colName = if (!lowerCase) rawColName else rawColName.toLowerCase + val typeString = Option(nodeToTypeString(colTypeNode)) + .map(HiveMetastoreTypes.toDataType) + .orNull CatalogColumn( name = cleanIdentifier(colName), - dataType = HiveMetastoreTypes.toDataType(nodeToTypeString(colTypeNode)), + dataType = typeString, nullable = true, comment.headOption.map(n => unescapeSQLString(n.text))) } From 539449215ebfc3df5d7b13fbd4808f7e37d20d77 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 17 Feb 2016 13:32:36 -0800 Subject: [PATCH 24/30] Change CatalogColumn#dataType to String This fixes a failing test in HiveCompatibilitySuite, where Spark was ignoring the character limit in varchar but Hive respected it. The issue was that we were converting Hive types to and from Spark DataType, and in the process losing the limit information. Instead of doing this conversion, we simply encode the data type as a string so we don't loes any information. This means less type-safety but the real fix is outside the scope of this patch. --- .../sql/catalyst/catalog/interface.scala | 5 +-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 12 ++++--- .../org/apache/spark/sql/hive/HiveQl.scala | 6 +--- .../sql/hive/client/HiveClientImpl.scala | 4 +-- .../hive/execution/CreateTableAsSelect.scala | 4 ++- .../hive/execution/CreateViewAsSelect.scala | 14 +++++--- .../sql/hive/HiveMetastoreCatalogSuite.scala | 9 ++--- .../apache/spark/sql/hive/HiveQlSuite.scala | 33 +++++++++---------- .../spark/sql/hive/client/VersionsSuite.scala | 2 +- 9 files changed, 47 insertions(+), 42 deletions(-) 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 99e0b7446ecbd..08c902bd960e1 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,7 +20,6 @@ package org.apache.spark.sql.catalyst.catalog import javax.annotation.Nullable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.types.DataType /** @@ -166,7 +165,9 @@ case class CatalogStorageFormat( */ case class CatalogColumn( name: String, - @Nullable dataType: DataType, // may be null when used to create views + // This may be null when used to create views. TODO: make this type-safe; this is left + // as a string due to issues in converting Hive varchars to and from SparkSQL strings. + @Nullable dataType: String, nullable: Boolean = true, comment: Option[String] = None) 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 0d73cb494f0b8..0c79544d87cc0 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 @@ -316,7 +316,9 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte specifiedDatabase = Option(dbName), name = tblName, tableType = tableType, - schema = relation.schema.map { f => CatalogColumn(f.name, f.dataType) }, + schema = relation.schema.map { f => + CatalogColumn(f.name, HiveMetastoreTypes.toMetastoreType(f.dataType)) + }, storage = CatalogStorageFormat( locationUri = Some(relation.paths.head), inputFormat = serde.inputFormat, @@ -605,7 +607,9 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val schema = if (table.schema.nonEmpty) { table.schema } else { - child.output.map { attr => CatalogColumn(attr.name, attr.dataType, attr.nullable) } + child.output.map { a => + CatalogColumn(a.name, HiveMetastoreTypes.toMetastoreType(a.dataType), a.nullable) + } } val desc = table.copy(schema = schema) @@ -768,7 +772,7 @@ private[hive] case class MetastoreRelation( override protected def otherCopyArgs: Seq[AnyRef] = table :: sqlContext :: Nil private def toHiveColumn(c: CatalogColumn): FieldSchema = { - new FieldSchema(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment.orNull) + new FieldSchema(c.name, c.dataType, c.comment.orNull) } @transient val hiveQlTable: HiveTable = { @@ -889,7 +893,7 @@ private[hive] case class MetastoreRelation( implicit class SchemaAttribute(f: CatalogColumn) { def toAttribute: AttributeReference = AttributeReference( f.name, - f.dataType, + HiveMetastoreTypes.toDataType(f.dataType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true )(qualifiers = Seq(alias.getOrElse(tableName))) 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 a4c3ef8473b8a..a49fe648fe59b 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 @@ -40,7 +40,6 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SparkQl import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper -import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.types._ import org.apache.spark.sql.AnalysisException @@ -688,12 +687,9 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging node.children.map(_.children).collect { case Token(rawColName, Nil) :: colTypeNode :: comment => val colName = if (!lowerCase) rawColName else rawColName.toLowerCase - val typeString = Option(nodeToTypeString(colTypeNode)) - .map(HiveMetastoreTypes.toDataType) - .orNull CatalogColumn( name = cleanIdentifier(colName), - dataType = typeString, + dataType = nodeToTypeString(colTypeNode), nullable = true, comment.headOption.map(n => unescapeSQLString(n.text))) } 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 1baa03870a5b2..6caf1c7820d28 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 @@ -616,13 +616,13 @@ private[hive] class HiveClientImpl( } private def toHiveColumn(c: CatalogColumn): FieldSchema = { - new FieldSchema(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), c.comment.orNull) + new FieldSchema(c.name, c.dataType, c.comment.orNull) } private def fromHiveColumn(hc: FieldSchema): CatalogColumn = { new CatalogColumn( name = hc.getName, - dataType = HiveMetastoreTypes.toDataType(hc.getType), + dataType = hc.getType, nullable = true, comment = Option(hc.getComment)) } 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 2aa5ed8cd891f..3f81c99c41e14 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 @@ -62,7 +62,9 @@ case class CreateTableAsSelect( val withSchema = if (withFormat.schema.isEmpty) { // Hive doesn't support specifying the column list for target table in CTAS // However we don't think SparkSQL should follow that. - tableDesc.copy(schema = query.output.map { c => CatalogColumn(c.name, c.dataType) }) + tableDesc.copy(schema = query.output.map { c => + CatalogColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType)) + }) } else { withFormat } 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 90d95b6ac33a5..b74c4868db681 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.{HiveContext, SQLBuilder} +import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveContext, SQLBuilder} /** * Create Hive view on non-hive-compatible tables by specifying schema ourselves instead of @@ -83,10 +83,16 @@ private[hive] case class CreateViewAsSelect( val viewSchema = { if (tableDesc.schema.isEmpty) { - childSchema.map { attr => CatalogColumn(attr.name, attr.dataType) } + childSchema.map { a => + CatalogColumn(a.name, HiveMetastoreTypes.toMetastoreType(a.dataType)) + } } else { - childSchema.zip(tableDesc.schema).map { case (attr, col) => - CatalogColumn(col.name, attr.dataType, nullable = true, col.comment) + childSchema.zip(tableDesc.schema).map { case (a, col) => + CatalogColumn( + col.name, + HiveMetastoreTypes.toMetastoreType(a.dataType), + nullable = true, + col.comment) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 87d61d8e93a95..f8764d4725f63 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -92,8 +92,7 @@ class DataSourceWithHiveMetastoreCatalogSuite val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map { a => HiveMetastoreTypes.toMetastoreType(a.dataType) } === - Seq("decimal(10,3)", "string")) + assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) @@ -125,8 +124,7 @@ class DataSourceWithHiveMetastoreCatalogSuite val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map { a => HiveMetastoreTypes.toMetastoreType(a.dataType) } === - Seq("decimal(10,3)", "string")) + assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) @@ -155,8 +153,7 @@ class DataSourceWithHiveMetastoreCatalogSuite val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map { a => HiveMetastoreTypes.toMetastoreType(a.dataType) } === - Seq("int", "string")) + assert(columns.map(_.dataType) === Seq("int", "string")) checkAnswer(table("t"), Row(1, "val_1")) assert(runSqlHive("SELECT * FROM t") === Seq("1\tval_1")) 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 d193ea115166d..3b3a907510134 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 @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, Catal import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.SimpleParserConf import org.apache.spark.sql.catalyst.plans.logical.Generate -import org.apache.spark.sql.types.{IntegerType, LongType, StringType} class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { val parser = new HiveQl(SimpleParserConf()) @@ -60,17 +59,17 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(desc.tableType == CatalogTableType.EXTERNAL_TABLE) assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == - CatalogColumn("viewtime", IntegerType) :: - CatalogColumn("userid", LongType) :: - CatalogColumn("page_url", StringType) :: - CatalogColumn("referrer_url", StringType) :: - CatalogColumn("ip", StringType, comment = Some("IP Address of the User")) :: - CatalogColumn("country", StringType, comment = Some("country of origination")) :: Nil) + CatalogColumn("viewtime", "int") :: + CatalogColumn("userid", "bigint") :: + CatalogColumn("page_url", "string") :: + CatalogColumn("referrer_url", "string") :: + CatalogColumn("ip", "string", comment = Some("IP Address of the User")) :: + CatalogColumn("country", "string", comment = Some("country of origination")) :: Nil) // TODO will be SQLText assert(desc.viewText == Option("This is the staging page view table")) assert(desc.partitionColumns == - CatalogColumn("dt", StringType, comment = Some("date type")) :: - CatalogColumn("hour", StringType, comment = Some("hour of the day")) :: Nil) + CatalogColumn("dt", "string", comment = Some("date type")) :: + CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) assert(desc.storage.serdeProperties == Map((serdeConstants.SERIALIZATION_FORMAT, "\054"), (serdeConstants.FIELD_DELIM, "\054"))) assert(desc.storage.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) @@ -106,17 +105,17 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(desc.tableType == CatalogTableType.EXTERNAL_TABLE) assert(desc.storage.locationUri == Some("/user/external/page_view")) assert(desc.schema == - CatalogColumn("viewtime", IntegerType) :: - CatalogColumn("userid", LongType) :: - CatalogColumn("page_url", StringType) :: - CatalogColumn("referrer_url", StringType) :: - CatalogColumn("ip", StringType, comment = Some("IP Address of the User")) :: - CatalogColumn("country", StringType, comment = Some("country of origination")) :: Nil) + CatalogColumn("viewtime", "int") :: + CatalogColumn("userid", "bigint") :: + CatalogColumn("page_url", "string") :: + CatalogColumn("referrer_url", "string") :: + CatalogColumn("ip", "string", comment = Some("IP Address of the User")) :: + CatalogColumn("country", "string", comment = Some("country of origination")) :: Nil) // TODO will be SQLText assert(desc.viewText == Option("This is the staging page view table")) assert(desc.partitionColumns == - CatalogColumn("dt", StringType, comment = Some("date type")) :: - CatalogColumn("hour", StringType, comment = Some("hour of the day")) :: Nil) + CatalogColumn("dt", "string", comment = Some("date type")) :: + CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) assert(desc.storage.serdeProperties == Map()) assert(desc.storage.inputFormat == Option("parquet.hive.DeprecatedParquetInputFormat")) assert(desc.storage.outputFormat == Option("parquet.hive.DeprecatedParquetOutputFormat")) 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 2b8548ed339cc..d850d522be297 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 @@ -127,7 +127,7 @@ class VersionsSuite extends SparkFunSuite with Logging { specifiedDatabase = Option("default"), name = "src", tableType = CatalogTableType.MANAGED_TABLE, - schema = Seq(CatalogColumn("key", IntegerType)), + schema = Seq(CatalogColumn("key", "int")), storage = CatalogStorageFormat( locationUri = None, inputFormat = Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), From fe295fb6899be00eb8a37eceb6c996cf0794ff2c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 17 Feb 2016 14:23:54 -0800 Subject: [PATCH 25/30] Fix style --- .../apache/spark/sql/hive/execution/CreateViewAsSelect.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b74c4868db681..2914d03749321 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveContext, SQLBuilder} +import org.apache.spark.sql.hive.{ HiveContext, HiveMetastoreTypes, SQLBuilder} /** * Create Hive view on non-hive-compatible tables by specifying schema ourselves instead of From 43e3c66057d37c45db7392c6793baeef05b05039 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 17 Feb 2016 16:32:59 -0800 Subject: [PATCH 26/30] Fix MetastoreDataSourcesSuite I missed one place where the data type was still a DataType, but not a string. --- .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 a9c5104a6b5e2..0c288bdf8a681 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 @@ -921,7 +921,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in Hive compatible format, we verify that // each column of the table is of native type StringType. assert(catalog.client.getTable("default", "not_skip_hive_metadata").schema - .forall(_.dataType == StringType)) + .forall(column => HiveMetastoreTypes.toDataType(column.dataType) == StringType)) catalog.createDataSourceTable( tableIdent = TableIdentifier("skip_hive_metadata"), @@ -935,6 +935,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in SparkSQL format, we verify that // the table has a column type as array of StringType. assert(catalog.client.getTable("default", "skip_hive_metadata").schema - .forall(_.dataType == ArrayType(StringType))) + .forall(column => HiveMetastoreTypes.toDataType(column.dataType) == ArrayType(StringType))) } } From 27656491561a918e4e5bec7f44ef946ef825dc19 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 17 Feb 2016 19:04:57 -0800 Subject: [PATCH 27/30] Add HiveCatalogSuite This suite extends the existing CatalogTestCases. Many tests needed to be modified significantly for Hive to work. Even after many hours spent on trying to make this work, there is still one that doesn't pass for some reason. In particular, I was not able to call "alterPartitions" on an existing Hive table as of this commit. That test is temporarily ignored for now. The rest of the tests added in this commit should pass. --- .../apache/spark/sql/AnalysisException.scala | 3 + .../spark/sql/catalyst/CatalystQl.scala | 1 - .../spark/sql/catalyst/analysis/Catalog.scala | 12 - .../analysis/NoSuchItemException.scala | 50 +++++ .../catalyst/catalog/InMemoryCatalog.scala | 6 - .../sql/catalyst/catalog/interface.scala | 18 ++ .../catalyst/catalog/CatalogTestCases.scala | 211 +++++++++++------- .../apache/spark/sql/hive/HiveCatalog.scala | 165 +++++++++++--- .../spark/sql/hive/client/HiveClient.scala | 17 +- .../sql/hive/client/HiveClientImpl.scala | 28 +-- .../spark/sql/hive/HiveCatalogSuite.scala | 49 ++++ 11 files changed, 406 insertions(+), 154 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index f9992185a4563..97f28fad62e43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql import org.apache.spark.annotation.DeveloperApi + +// TODO: don't swallow original stack trace if it exists + /** * :: DeveloperApi :: * Thrown when a query fails to analyze, usually because the query itself is invalid. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 1a50e8d3ede20..8099751900a42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -52,7 +52,6 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends case e: MatchError => throw e case e: AnalysisException => throw e case e: Exception => - // TODO: don't swallow original stack trace throw new AnalysisException(e.getMessage) case e: NotImplementedError => throw new AnalysisException( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index b825164bcf935..af6f7b0930c39 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -26,18 +26,6 @@ import org.apache.spark.sql.catalyst.{CatalystConf, EmptyConf, TableIdentifier} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} -/** - * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the - * exception as an [[AnalysisException]] with the correct position information. - */ -abstract class NoSuchItemException extends Exception - -class NoSuchFunctionException extends NoSuchItemException -class NoSuchPartitionException extends NoSuchItemException -class NoSuchTableException extends NoSuchItemException -class NoSuchDatabaseException extends NoSuchItemException - - /** * An interface for looking up relations by name. Used by an [[Analyzer]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala new file mode 100644 index 0000000000000..221cd68231906 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.catalog.Catalog.TablePartitionSpec + + +/** + * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception + * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. + */ +abstract class NoSuchItemException extends Exception { override def getMessage: String } + +class NoSuchDatabaseException(db: String) extends NoSuchItemException { + override def getMessage: String = s"Database $db not found" +} + +class NoSuchTableException(db: String, table: String) extends NoSuchItemException { + override def getMessage: String = s"Table $table not found in database $db" +} + +class NoSuchPartitionException( + db: String, + table: String, + spec: TablePartitionSpec) + extends NoSuchItemException { + + override def getMessage: String = { + s"Partition not found in table $table database $db:\n" + spec.mkString("\n") + } +} + +class NoSuchFunctionException(db: String, func: String) extends NoSuchItemException { + override def getMessage: String = s"Function $func not found in database $db" +} 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 57617d835ae92..72fa49f3ececc 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 @@ -62,12 +62,6 @@ class InMemoryCatalog extends Catalog { catalog(db).tables(table).partitions.contains(spec) } - private def requireDbExists(db: String): Unit = { - if (!catalog.contains(db)) { - throw new AnalysisException(s"Database $db does not exist") - } - } - private def requireFunctionExists(db: String, funcName: String): Unit = { if (!existsFunction(db, funcName)) { throw new AnalysisException(s"Function $funcName does not exist in $db database") 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 08c902bd960e1..c5bfc8dbd0e79 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 @@ -33,6 +33,12 @@ import org.apache.spark.sql.AnalysisException abstract class Catalog { import Catalog._ + protected def requireDbExists(db: String): Unit = { + if (!databaseExists(db)) { + throw new AnalysisException(s"Database $db does not exist") + } + } + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -44,6 +50,9 @@ abstract class Catalog { /** * Alter a database whose name matches the one specified in `dbDefinition`, * assuming the database exists. + * + * Note: If the underlying implementation does not support altering a certain field, + * this becomes a no-op. */ def alterDatabase(dbDefinition: CatalogDatabase): Unit @@ -68,6 +77,9 @@ abstract class Catalog { /** * Alter a table whose name that 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 @@ -106,6 +118,9 @@ abstract class Catalog { /** * 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, @@ -130,6 +145,9 @@ abstract class Catalog { /** * 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(db: String, funcDefinition: CatalogFunction): Unit diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogTestCases.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogTestCases.scala index 4efdc5781137f..f0b8616b6a5aa 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.catalog +import org.scalatest.BeforeAndAfterEach + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException @@ -26,18 +28,38 @@ import org.apache.spark.sql.AnalysisException * * Implementations of the [[Catalog]] interface can create test suites by extending this. */ -abstract class CatalogTestCases extends SparkFunSuite { - private val storageFormat = CatalogStorageFormat(None, Some("z"), Some("y"), Some("x"), Map()) - private val part1 = CatalogTablePartition(Map("a" -> "1"), storageFormat) - private val part2 = CatalogTablePartition(Map("b" -> "2"), storageFormat) - private val part3 = CatalogTablePartition(Map("c" -> "3"), storageFormat) +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 def resetState(): Unit = { } protected def newEmptyCatalog(): Catalog + // Clear all state after each test + override def afterEach(): Unit = { + try { + resetState() + } finally { + super.afterEach() + } + } + /** * Creates a basic catalog, with the following structure: * + * default * db1 * db2 * - tbl1 @@ -48,10 +70,12 @@ abstract class CatalogTestCases extends SparkFunSuite { */ private def newBasicCatalog(): Catalog = { 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"), ignoreIfExists = false) - catalog.createTable("db2", newTable("tbl2"), 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")) catalog @@ -59,37 +83,51 @@ abstract class CatalogTestCases extends SparkFunSuite { private def newFunc(): CatalogFunction = CatalogFunction("funcname", funcClass) - private def newDb(name: String = "default"): CatalogDatabase = - CatalogDatabase(name, name + " description", "uri", Map.empty) + private def newDb(name: String): CatalogDatabase = { + CatalogDatabase(name, name + " description", newUriForDatabase(), Map.empty) + } - private def newTable(name: String): CatalogTable = + private def newTable(name: String, db: String): CatalogTable = { CatalogTable( - specifiedDatabase = Some("db"), + specifiedDatabase = Some(db), name = name, tableType = CatalogTableType.EXTERNAL_TABLE, - storage = CatalogStorageFormat( - locationUri = Some("just works"), - inputFormat = Some("this part will change"), - outputFormat = Some("just one line"), - serde = Some("if I stopped the stream"), - serdeProperties = Map.empty - ), - schema = Seq.empty) + storage = storageFormat, + schema = Seq(CatalogColumn("col1", "int"), CatalogColumn("col2", "string")), + partitionColumns = Seq(CatalogColumn("a", "int"), CatalogColumn("b", "string"))) + } private def newFunc(name: String): CatalogFunction = CatalogFunction(name, funcClass) + /** + * Whether the catalog's table partitions equals the ones given. + * Note: Hive sets some random serde things, so we just compare the specs here. + */ + private def catalogPartitionsEqual( + catalog: Catalog, + db: String, + table: String, + parts: Seq[CatalogTablePartition]): Boolean = { + catalog.listPartitions(db, table).map(_.spec).toSet == parts.map(_.spec).toSet + } + + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- - test("basic create, drop and list databases") { + test("basic create and list databases") { val catalog = newEmptyCatalog() - catalog.createDatabase(newDb(), ignoreIfExists = false) - assert(catalog.listDatabases().toSet == Set("default")) - catalog.createDatabase(newDb("default2"), ignoreIfExists = false) - assert(catalog.listDatabases().toSet == Set("default", "default2")) + catalog.createDatabase(newDb("default"), ignoreIfExists = true) assert(catalog.databaseExists("default")) - assert(catalog.databaseExists("default2")) + 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")) } @@ -105,7 +143,7 @@ abstract class CatalogTestCases extends SparkFunSuite { test("list databases without pattern") { val catalog = newBasicCatalog() - assert(catalog.listDatabases().toSet == Set("db1", "db2")) + assert(catalog.listDatabases().toSet == Set("default", "db1", "db2")) } test("list databases with pattern") { @@ -119,7 +157,7 @@ abstract class CatalogTestCases extends SparkFunSuite { test("drop database") { val catalog = newBasicCatalog() catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false) - assert(catalog.listDatabases().toSet == Set("db2")) + assert(catalog.listDatabases().toSet == Set("default", "db2")) } test("drop database when the database is not empty") { @@ -130,6 +168,7 @@ abstract class CatalogTestCases extends SparkFunSuite { intercept[AnalysisException] { catalog1.dropDatabase("db2", ignoreIfNotExists = false, cascade = false) } + resetState() // Throw exception if there are tables left val catalog2 = newBasicCatalog() @@ -137,11 +176,12 @@ abstract class CatalogTestCases extends SparkFunSuite { intercept[AnalysisException] { catalog2.dropDatabase("db2", ignoreIfNotExists = false, cascade = false) } + resetState() // When cascade is true, it should drop them val catalog3 = newBasicCatalog() catalog3.dropDatabase("db2", ignoreIfNotExists = false, cascade = true) - assert(catalog3.listDatabases().toSet == Set("db1")) + assert(catalog3.listDatabases().toSet == Set("default", "db1")) } test("drop database when the database does not exist") { @@ -156,13 +196,19 @@ abstract class CatalogTestCases extends SparkFunSuite { test("alter database") { val catalog = newBasicCatalog() - catalog.alterDatabase(CatalogDatabase("db1", "new description", "lll", Map.empty)) - assert(catalog.getDatabase("db1").description == "new description") + 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") { intercept[AnalysisException] { - newBasicCatalog().alterDatabase(CatalogDatabase("no_db", "ddd", "lll", Map.empty)) + newBasicCatalog().alterDatabase(newDb("does_not_exist")) } } @@ -177,61 +223,56 @@ abstract class CatalogTestCases extends SparkFunSuite { assert(catalog.listTables("db2").toSet == Set("tbl2")) } - test("drop table when database / table does not exist") { + test("drop table when database/table does not exist") { val catalog = newBasicCatalog() - // Should always throw exception when the database does not exist intercept[AnalysisException] { catalog.dropTable("unknown_db", "unknown_table", ignoreIfNotExists = false) } - intercept[AnalysisException] { catalog.dropTable("unknown_db", "unknown_table", ignoreIfNotExists = true) } - // Should throw exception when the table does not exist, if ignoreIfNotExists is false intercept[AnalysisException] { catalog.dropTable("db2", "unknown_table", ignoreIfNotExists = false) } - catalog.dropTable("db2", "unknown_table", ignoreIfNotExists = true) } test("rename table") { val catalog = newBasicCatalog() - assert(catalog.listTables("db2").toSet == Set("tbl1", "tbl2")) catalog.renameTable("db2", "tbl1", "tblone") assert(catalog.listTables("db2").toSet == Set("tblone", "tbl2")) } - test("rename table when database / table does not exist") { + test("rename table when database/table does not exist") { val catalog = newBasicCatalog() - - intercept[AnalysisException] { // Throw exception when the database does not exist + intercept[AnalysisException] { catalog.renameTable("unknown_db", "unknown_table", "unknown_table") } - - intercept[AnalysisException] { // Throw exception when the table does not exist + intercept[AnalysisException] { catalog.renameTable("db2", "unknown_table", "unknown_table") } } test("alter table") { val catalog = newBasicCatalog() - catalog.alterTable("db2", newTable("tbl1").copy(createTime = 10)) - assert(catalog.getTable("db2", "tbl1").createTime == 10) + val tbl1 = catalog.getTable("db2", "tbl1") + catalog.alterTable("db2", tbl1.copy(properties = Map("toh" -> "frem"))) + val newTbl1 = catalog.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") { + test("alter table when database/table does not exist") { val catalog = newBasicCatalog() - - intercept[AnalysisException] { // Throw exception when the database does not exist - catalog.alterTable("unknown_db", newTable("unknown_table")) + intercept[AnalysisException] { + catalog.alterTable("unknown_db", newTable("tbl1", "unknown_db")) } - - intercept[AnalysisException] { // Throw exception when the table does not exist - catalog.alterTable("db2", newTable("unknown_table")) + intercept[AnalysisException] { + catalog.alterTable("db2", newTable("unknown_table", "db2")) } } @@ -239,12 +280,11 @@ abstract class CatalogTestCases extends SparkFunSuite { assert(newBasicCatalog().getTable("db2", "tbl1").name == "tbl1") } - test("get table when database / table does not exist") { + test("get table when database/table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { catalog.getTable("unknown_db", "unknown_table") } - intercept[AnalysisException] { catalog.getTable("db2", "unknown_table") } @@ -258,10 +298,7 @@ abstract class CatalogTestCases extends SparkFunSuite { test("list tables with pattern") { val catalog = newBasicCatalog() - - // Test when database does not exist intercept[AnalysisException] { catalog.listTables("unknown_db") } - assert(catalog.listTables("db1", "*").toSet == Set.empty) assert(catalog.listTables("db2", "*").toSet == Set("tbl1", "tbl2")) assert(catalog.listTables("db2", "tbl*").toSet == Set("tbl1", "tbl2")) @@ -275,12 +312,12 @@ abstract class CatalogTestCases extends SparkFunSuite { test("basic create and list partitions") { val catalog = newEmptyCatalog() catalog.createDatabase(newDb("mydb"), ignoreIfExists = false) - catalog.createTable("mydb", newTable("mytbl"), ignoreIfExists = false) - catalog.createPartitions("mydb", "mytbl", Seq(part1, part2), ignoreIfExists = false) - assert(catalog.listPartitions("mydb", "mytbl").toSet == Set(part1, part2)) + catalog.createTable("mydb", newTable("tbl", "mydb"), ignoreIfExists = false) + catalog.createPartitions("mydb", "tbl", Seq(part1, part2), ignoreIfExists = false) + assert(catalogPartitionsEqual(catalog, "mydb", "tbl", Seq(part1, part2))) } - test("create partitions when database / table does not exist") { + test("create partitions when database/table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { catalog.createPartitions("does_not_exist", "tbl1", Seq(), ignoreIfExists = false) @@ -300,16 +337,17 @@ abstract class CatalogTestCases extends SparkFunSuite { test("drop partitions") { val catalog = newBasicCatalog() - assert(catalog.listPartitions("db2", "tbl2").toSet == Set(part1, part2)) + assert(catalogPartitionsEqual(catalog, "db2", "tbl2", Seq(part1, part2))) catalog.dropPartitions("db2", "tbl2", Seq(part1.spec), ignoreIfNotExists = false) - assert(catalog.listPartitions("db2", "tbl2").toSet == Set(part2)) + assert(catalogPartitionsEqual(catalog, "db2", "tbl2", Seq(part2))) + resetState() val catalog2 = newBasicCatalog() - assert(catalog2.listPartitions("db2", "tbl2").toSet == Set(part1, part2)) + assert(catalogPartitionsEqual(catalog2, "db2", "tbl2", Seq(part1, part2))) catalog2.dropPartitions("db2", "tbl2", Seq(part1.spec, part2.spec), ignoreIfNotExists = false) assert(catalog2.listPartitions("db2", "tbl2").isEmpty) } - test("drop partitions when database / table does not exist") { + test("drop partitions when database/table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { catalog.dropPartitions("does_not_exist", "tbl1", Seq(), ignoreIfNotExists = false) @@ -329,14 +367,14 @@ abstract class CatalogTestCases extends SparkFunSuite { test("get partition") { val catalog = newBasicCatalog() - assert(catalog.getPartition("db2", "tbl2", part1.spec) == part1) - assert(catalog.getPartition("db2", "tbl2", part2.spec) == part2) + assert(catalog.getPartition("db2", "tbl2", part1.spec).spec == part1.spec) + assert(catalog.getPartition("db2", "tbl2", part2.spec).spec == part2.spec) intercept[AnalysisException] { catalog.getPartition("db2", "tbl1", part3.spec) } } - test("get partition when database / table does not exist") { + test("get partition when database/table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { catalog.getPartition("does_not_exist", "tbl1", part1.spec) @@ -348,17 +386,18 @@ abstract class CatalogTestCases extends SparkFunSuite { test("rename partitions") { val catalog = newBasicCatalog() - val newPart1 = part1.copy(spec = Map("x" -> "10")) - val newPart2 = part2.copy(spec = Map("y" -> "12")) + 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", "tbl2", Seq(part1.spec, part2.spec), newSpecs) - assert(catalog.getPartition("db2", "tbl2", newPart1.spec) === newPart1) - assert(catalog.getPartition("db2", "tbl2", newPart2.spec) === newPart2) + assert(catalog.getPartition("db2", "tbl2", newPart1.spec).spec === newPart1.spec) + assert(catalog.getPartition("db2", "tbl2", newPart2.spec).spec === newPart2.spec) + // The old partitions should no longer exist intercept[AnalysisException] { catalog.getPartition("db2", "tbl2", part1.spec) } intercept[AnalysisException] { catalog.getPartition("db2", "tbl2", part2.spec) } } - test("rename partitions when database / table does not exist") { + test("rename partitions when database/table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { catalog.renamePartitions("does_not_exist", "tbl1", Seq(part1.spec), Seq(part2.spec)) @@ -368,25 +407,31 @@ abstract class CatalogTestCases extends SparkFunSuite { } } - test("alter partitions") { + // TODO: figure out how to fix me for Hive, which currently fails with the most helpful + // error message in the world: "Unable to alter partition. alter is not possible" + ignore("alter partitions") { val catalog = newBasicCatalog() - val newPart1 = part1.copy(storage = storageFormat.copy(locationUri = Some("usa"))) - val newPart2 = part2.copy(storage = storageFormat.copy(locationUri = Some("china"))) - assert(catalog.getPartition("db2", "tbl2", part1.spec).storage.locationUri.isEmpty) - assert(catalog.getPartition("db2", "tbl2", part2.spec).storage.locationUri.isEmpty) + val oldPart1 = catalog.getPartition("db2", "tbl2", part1.spec) + val oldPart2 = catalog.getPartition("db2", "tbl2", part2.spec) // alter but keep spec the same - catalog.alterPartitions("db2", "tbl2", Seq(newPart1, newPart2)) - assert(catalog.getPartition("db2", "tbl2", part1.spec).storage.locationUri === Some("usa")) - assert(catalog.getPartition("db2", "tbl2", part2.spec).storage.locationUri === Some("china")) + catalog.alterPartitions("db2", "tbl2", Seq( + part1.copy(storage = storageFormat.copy(locationUri = Some("usa"))), + part2.copy(storage = storageFormat.copy(locationUri = Some("china"))))) + val newPart1 = catalog.getPartition("db2", "tbl2", part1.spec) + val newPart2 = catalog.getPartition("db2", "tbl2", part2.spec) + assert(newPart1.storage.locationUri != oldPart1.storage.locationUri) + assert(newPart2.storage.locationUri != oldPart2.storage.locationUri) + assert(newPart1.storage.locationUri == Some("usa")) + assert(newPart2.storage.locationUri == Some("china")) // alter but change spec, should fail because new partition specs do not exist yet - val badPart1 = part1.copy(spec = Map("k" -> "v")) - val badPart2 = part2.copy(spec = Map("k" -> "v")) + 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", "tbl2", Seq(badPart1, badPart2)) } } - test("alter partitions when database / table does not exist") { + test("alter partitions when database/table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { catalog.alterPartitions("does_not_exist", "tbl1", Seq(part1)) 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 c5044ccb764eb..bb9920e16f0b7 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 @@ -17,60 +17,121 @@ package org.apache.spark.sql.hive +import scala.util.control.NonFatal + +import org.apache.hadoop.hive.ql.metadata.HiveException +import org.apache.thrift.TException + +import org.apache.spark.Logging +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 /** * A persistent implementation of the system catalog using Hive. - * * All public methods must be synchronized for thread-safety. */ -private[spark] class HiveCatalog(client: HiveClient) extends Catalog { +private[spark] class HiveCatalog(client: HiveClient) extends Catalog with Logging { import Catalog._ + // Exceptions thrown by the hive client that we would like to wrap + private val clientExceptions = Set( + classOf[HiveException].getCanonicalName, + classOf[TException].getCanonicalName) + /** - * Assert that the provided database matches the one specified in the table. + * Whether this is an exception thrown by the hive client that should be wrapped. + * + * Due to classloader isolation issues, pattern matching won't work here so we need + * to compare the canonical names of the exceptions, which we assume to be stable. */ + private def isClientException(e: Throwable): Boolean = { + var temp: Class[_] = e.getClass + var found = false + while (temp != null && !found) { + found = clientExceptions.contains(temp.getCanonicalName) + temp = temp.getSuperclass + } + found + } + + /** + * Run some code involving `client` in a [[synchronized]] block and wrap certain + * exceptions thrown in the process in [[AnalysisException]]. + */ + private def withClient[T](body: => T): T = synchronized { + try { + body + } catch { + case e: NoSuchItemException => + throw new AnalysisException(e.getMessage) + case NonFatal(e) if isClientException(e) => + throw new AnalysisException(e.getClass.getCanonicalName + ": " + e.getMessage) + } + } + private def requireDbMatches(db: String, table: CatalogTable): Unit = { - require(table.specifiedDatabase == Some(db), - "provided database does not much the one specified in the table definition") + if (table.specifiedDatabase != 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")})") + } + } + + private def requireTableExists(db: String, table: String): Unit = { + withClient { getTable(db, table) } } + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- override def createDatabase( dbDefinition: CatalogDatabase, - ignoreIfExists: Boolean): Unit = synchronized { + ignoreIfExists: Boolean): Unit = withClient { client.createDatabase(dbDefinition, ignoreIfExists) } override def dropDatabase( db: String, ignoreIfNotExists: Boolean, - cascade: Boolean): Unit = synchronized { + cascade: Boolean): Unit = withClient { client.dropDatabase(db, ignoreIfNotExists, cascade) } - override def alterDatabase(dbDefinition: CatalogDatabase): Unit = synchronized { - client.alterDatabase(dbDefinition) + /** + * Alter a database whose name matches the one specified in `dbDefinition`, + * assuming the database exists. + * + * Note: As of now, this only supports altering database properties! + */ + override def alterDatabase(dbDefinition: CatalogDatabase): Unit = withClient { + val existingDb = getDatabase(dbDefinition.name) + if (existingDb.properties == dbDefinition.properties) { + logWarning(s"Ignoring request to alter database ${dbDefinition.name} because " + + s"the provided database properties are the same as the old ones. Hive does not " + + s"currently support altering other database fields.") + } else { + client.alterDatabase(dbDefinition) + } } - override def getDatabase(db: String): CatalogDatabase = synchronized { + override def getDatabase(db: String): CatalogDatabase = withClient { client.getDatabase(db) } - override def databaseExists(db: String): Boolean = synchronized { + override def databaseExists(db: String): Boolean = withClient { client.getDatabaseOption(db).isDefined } - override def listDatabases(): Seq[String] = synchronized { + override def listDatabases(): Seq[String] = withClient { client.listDatabases("*") } - override def listDatabases(pattern: String): Seq[String] = synchronized { + override def listDatabases(pattern: String): Seq[String] = withClient { client.listDatabases(pattern) } @@ -81,7 +142,8 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def createTable( db: String, tableDefinition: CatalogTable, - ignoreIfExists: Boolean): Unit = synchronized { + ignoreIfExists: Boolean): Unit = withClient { + requireDbExists(db) requireDbMatches(db, tableDefinition) client.createTable(tableDefinition, ignoreIfExists) } @@ -89,29 +151,41 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def dropTable( db: String, table: String, - ignoreIfNotExists: Boolean): Unit = synchronized { + ignoreIfNotExists: Boolean): Unit = withClient { + requireDbExists(db) client.dropTable(db, table, ignoreIfNotExists) } - override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized { + override def renameTable(db: String, oldName: String, newName: String): Unit = withClient { val newTable = client.getTable(db, oldName).copy(name = newName) client.alterTable(oldName, newTable) } - override def alterTable(db: String, tableDefinition: CatalogTable): Unit = synchronized { + /** + * Alter a table whose name that matches the one specified in `tableDefinition`, + * assuming the table exists. + * + * Note: As of now, this only supports altering table properties, serde properties, + * and num buckets! + */ + override def alterTable(db: String, tableDefinition: CatalogTable): Unit = withClient { requireDbMatches(db, tableDefinition) + requireTableExists(db, tableDefinition.name) client.alterTable(tableDefinition) } - override def getTable(db: String, table: String): CatalogTable = synchronized { + override def getTable(db: String, table: String): CatalogTable = withClient { + requireDbExists(db) client.getTable(db, table) } - override def listTables(db: String): Seq[String] = synchronized { + override def listTables(db: String): Seq[String] = withClient { + requireDbExists(db) client.listTables(db) } - override def listTables(db: String, pattern: String): Seq[String] = synchronized { + override def listTables(db: String, pattern: String): Seq[String] = withClient { + requireDbExists(db) client.listTables(db, pattern) } @@ -123,7 +197,8 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, table: String, parts: Seq[CatalogTablePartition], - ignoreIfExists: Boolean): Unit = synchronized { + ignoreIfExists: Boolean): Unit = withClient { + requireTableExists(db, table) client.createPartitions(db, table, parts, ignoreIfExists) } @@ -131,35 +206,59 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { db: String, table: String, parts: Seq[TablePartitionSpec], - ignoreIfNotExists: Boolean): Unit = synchronized { - client.dropPartitions(db, table, parts, ignoreIfNotExists) + ignoreIfNotExists: Boolean): Unit = withClient { + requireTableExists(db, table) + // Note: Unfortunately Hive does not currently support `ignoreIfNotExists` so we + // need to implement it here ourselves. This is currently somewhat expensive because + // we make multiple synchronous calls to Hive for each partition we want to drop. + val partsToDrop = + if (ignoreIfNotExists) { + parts.filter { spec => + try { + getPartition(db, table, spec) + true + } catch { + // Filter out the partitions that do not actually exist + case _: AnalysisException => false + } + } + } else { + parts + } + if (partsToDrop.nonEmpty) { + client.dropPartitions(db, table, partsToDrop) + } } override def renamePartitions( db: String, table: String, specs: Seq[TablePartitionSpec], - newSpecs: Seq[TablePartitionSpec]): Unit = synchronized { + newSpecs: Seq[TablePartitionSpec]): Unit = withClient { + requireTableExists(db, table) client.renamePartitions(db, table, specs, newSpecs) } override def alterPartitions( db: String, table: String, - newParts: Seq[CatalogTablePartition]): Unit = synchronized { + newParts: Seq[CatalogTablePartition]): Unit = withClient { + requireTableExists(db, table) client.alterPartitions(db, table, newParts) } override def getPartition( db: String, table: String, - spec: TablePartitionSpec): CatalogTablePartition = synchronized { + spec: TablePartitionSpec): CatalogTablePartition = withClient { + requireTableExists(db, table) client.getPartition(db, table, spec) } override def listPartitions( db: String, - table: String): Seq[CatalogTablePartition] = synchronized { + table: String): Seq[CatalogTablePartition] = withClient { + requireTableExists(db, table) client.getAllPartitions(db, table) } @@ -169,27 +268,27 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog { override def createFunction( db: String, - funcDefinition: CatalogFunction): Unit = synchronized { + funcDefinition: CatalogFunction): Unit = withClient { client.createFunction(db, funcDefinition) } - override def dropFunction(db: String, name: String): Unit = synchronized { + override def dropFunction(db: String, name: String): Unit = withClient { client.dropFunction(db, name) } - override def renameFunction(db: String, oldName: String, newName: String): Unit = synchronized { + override def renameFunction(db: String, oldName: String, newName: String): Unit = withClient { client.renameFunction(db, oldName, newName) } - override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = synchronized { + override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = withClient { client.alterFunction(db, funcDefinition) } - override def getFunction(db: String, funcName: String): CatalogFunction = synchronized { + override def getFunction(db: String, funcName: String): CatalogFunction = withClient { client.getFunction(db, funcName) } - override def listFunctions(db: String, pattern: String): Seq[String] = synchronized { + override def listFunctions(db: String, pattern: String): Seq[String] = withClient { client.listFunctions(db, pattern) } 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 4b789d8a31842..c7527068fd5e4 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 @@ -61,7 +61,7 @@ private[hive] trait HiveClient { /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ final def getDatabase(name: String): CatalogDatabase = { - getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException) + getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException(name)) } /** Returns the metadata for a given database, or None if it doesn't exist. */ @@ -72,7 +72,7 @@ private[hive] trait HiveClient { /** Returns the specified table, or throws [[NoSuchTableException]]. */ final def getTable(dbName: String, tableName: String): CatalogTable = { - getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException) + getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException(dbName, tableName)) } /** Returns the metadata for the specified table or None if it doens't exist. */ @@ -124,12 +124,15 @@ private[hive] trait HiveClient { /** * Drop one or many partitions in the given table. + * + * Note: Unfortunately, Hive does not currently provide a way to ignore this call if the + * partitions do not already exist. The seemingly relevant flag `ifExists` in + * [[org.apache.hadoop.hive.metastore.PartitionDropOptions]] is not read anywhere. */ def dropPartitions( db: String, table: String, - specs: Seq[Catalog.TablePartitionSpec], - ignoreIfNotExists: Boolean): Unit + specs: Seq[Catalog.TablePartitionSpec]): Unit /** * Rename one or many existing table partitions, assuming they exist. @@ -154,7 +157,9 @@ private[hive] trait HiveClient { dbName: String, tableName: String, spec: Catalog.TablePartitionSpec): CatalogTablePartition = { - getPartitionOption(dbName, tableName, spec).getOrElse(throw new NoSuchPartitionException) + getPartitionOption(dbName, tableName, spec).getOrElse { + throw new NoSuchPartitionException(dbName, tableName, spec) + } } /** Returns the specified partition or None if it does not exist. */ @@ -224,7 +229,7 @@ private[hive] trait HiveClient { /** Return an existing function in the database, assuming it exists. */ final def getFunction(db: String, name: String): CatalogFunction = { - getFunctionOption(db, name).getOrElse(throw new NoSuchFunctionException) + getFunctionOption(db, name).getOrElse(throw new NoSuchFunctionException(db, name)) } /** Return an existing function in the database, or None if it doesn't exist. */ 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 6caf1c7820d28..f89c3bce2cd9c 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 @@ -24,9 +24,9 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.{PartitionDropOptions, TableType => HiveTableType} +import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, - FieldSchema, Function => HiveFunction, ResourceUri} + FieldSchema, Function => HiveFunction, FunctionType, PrincipalType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.AddPartitionDesc @@ -40,7 +40,6 @@ import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPa import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.hive.HiveMetastoreTypes import org.apache.spark.util.{CircularBuffer, Utils} /** @@ -237,7 +236,7 @@ private[hive] class HiveClientImpl( if (getDatabaseOption(databaseName).isDefined) { state.setCurrentDatabase(databaseName) } else { - throw new NoSuchDatabaseException + throw new NoSuchDatabaseException(databaseName) } } @@ -358,14 +357,9 @@ private[hive] class HiveClientImpl( override def dropPartitions( db: String, table: String, - specs: Seq[Catalog.TablePartitionSpec], - ignoreIfNotExists: Boolean): Unit = withHiveState { + specs: Seq[Catalog.TablePartitionSpec]): Unit = withHiveState { // TODO: figure out how to drop multiple partitions in one call - specs.foreach { s => - val dropOptions = new PartitionDropOptions - dropOptions.ifExists = ignoreIfNotExists - client.dropPartition(db, table, s.values.toList.asJava, dropOptions) - } + specs.foreach { s => client.dropPartition(db, table, s.values.toList.asJava, true) } } override def renamePartitions( @@ -379,7 +373,7 @@ private[hive] class HiveClientImpl( specs.zip(newSpecs).foreach { case (oldSpec, newSpec) => val hivePart = getPartitionOption(catalogTable, oldSpec) .map { p => toHivePartition(p.copy(spec = newSpec), hiveTable) } - .getOrElse { throw new NoSuchPartitionException } + .getOrElse { throw new NoSuchPartitionException(db, table, oldSpec) } client.renamePartition(hiveTable, oldSpec.asJava, hivePart) } } @@ -608,7 +602,15 @@ private[hive] class HiveClientImpl( .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { - new HiveFunction(f.name, db, f.className, null, null, -1, null, List.empty[ResourceUri].asJava) + new HiveFunction( + f.name, + db, + f.className, + null, + PrincipalType.USER, + (System.currentTimeMillis / 1000).toInt, + FunctionType.JAVA, + List.empty[ResourceUri].asJava) } private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { 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 new file mode 100644 index 0000000000000..f73e7e2351447 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.util.VersionInfo + +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.hive.client.{HiveClient, IsolatedClientLoader} +import org.apache.spark.util.Utils + + +/** + * Test suite for the [[HiveCatalog]]. + */ +class HiveCatalogSuite extends CatalogTestCases { + + private val client: HiveClient = { + IsolatedClientLoader.forVersion( + hiveMetastoreVersion = HiveContext.hiveExecutionVersion, + hadoopVersion = VersionInfo.getVersion).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 def resetState(): Unit = client.reset() + + protected override def newEmptyCatalog(): Catalog = new HiveCatalog(client) + +} From cb288da8f0eff5f4a4703d83cd12262a7b3c3340 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 18 Feb 2016 15:19:21 -0800 Subject: [PATCH 28/30] Miscellaneous clean ups --- .../sql/catalyst/catalog/interface.scala | 10 +-- .../catalyst/catalog/CatalogTestCases.scala | 2 +- .../apache/spark/sql/hive/HiveCatalog.scala | 10 +-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 67 ++++--------------- .../org/apache/spark/sql/hive/HiveQl.scala | 4 +- .../spark/sql/hive/client/HiveClient.scala | 12 +++- .../sql/hive/client/HiveClientImpl.scala | 6 +- .../apache/spark/sql/hive/HiveQlSuite.scala | 22 +++--- 8 files changed, 46 insertions(+), 87 deletions(-) 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 c5bfc8dbd0e79..7e0071ced233f 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 @@ -220,14 +220,10 @@ case class CatalogTable( viewOriginalText: Option[String] = None, viewText: Option[String] = None) { - require( - tableType == CatalogTableType.EXTERNAL_TABLE || - tableType == CatalogTableType.INDEX_TABLE || - tableType == CatalogTableType.MANAGED_TABLE || - tableType == CatalogTableType.VIRTUAL_VIEW) - /** Return the database this table was specified to belong to, assuming it exists. */ - def database: String = specifiedDatabase.getOrElse(sys.error("database not resolved")) + def database: String = specifiedDatabase.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" 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 f0b8616b6a5aa..39f09ffa78262 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 @@ -100,7 +100,7 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { private def newFunc(name: String): CatalogFunction = CatalogFunction(name, funcClass) /** - * Whether the catalog's table partitions equals the ones given. + * 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( 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 bb9920e16f0b7..8bb6e91b49c36 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 @@ -111,12 +111,11 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog with Loggin override def alterDatabase(dbDefinition: CatalogDatabase): Unit = withClient { val existingDb = getDatabase(dbDefinition.name) if (existingDb.properties == dbDefinition.properties) { - logWarning(s"Ignoring request to alter database ${dbDefinition.name} because " + + logWarning(s"Request to alter database ${dbDefinition.name} is a no-op because " + s"the provided database properties are the same as the old ones. Hive does not " + s"currently support altering other database fields.") - } else { - client.alterDatabase(dbDefinition) } + client.alterDatabase(dbDefinition) } override def getDatabase(db: String): CatalogDatabase = withClient { @@ -175,7 +174,6 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog with Loggin } override def getTable(db: String, table: String): CatalogTable = withClient { - requireDbExists(db) client.getTable(db, table) } @@ -235,7 +233,6 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog with Loggin table: String, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = withClient { - requireTableExists(db, table) client.renamePartitions(db, table, specs, newSpecs) } @@ -243,7 +240,6 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog with Loggin db: String, table: String, newParts: Seq[CatalogTablePartition]): Unit = withClient { - requireTableExists(db, table) client.alterPartitions(db, table, newParts) } @@ -251,14 +247,12 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog with Loggin db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition = withClient { - requireTableExists(db, table) client.getPartition(db, table, spec) } override def listPartitions( db: String, table: String): Seq[CatalogTablePartition] = withClient { - requireTableExists(db, table) client.getAllPartitions(db, table) } 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 0c79544d87cc0..56dabe7f3bc3e 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 @@ -311,14 +311,10 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte assert(partitionColumns.isEmpty) assert(relation.partitionColumns.isEmpty) - // TODO: We need to place the SQL string in the view text here. CatalogTable( specifiedDatabase = Option(dbName), name = tblName, tableType = tableType, - schema = relation.schema.map { f => - CatalogColumn(f.name, HiveMetastoreTypes.toMetastoreType(f.dataType)) - }, storage = CatalogStorageFormat( locationUri = Some(relation.paths.head), inputFormat = serde.inputFormat, @@ -326,7 +322,11 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte serde = serde.serde, serdeProperties = options ), - properties = tableProperties.toMap) + schema = relation.schema.map { f => + CatalogColumn(f.name, HiveMetastoreTypes.toMetastoreType(f.dataType)) + }, + properties = tableProperties.toMap, + viewText = None) // TODO: We need to place the SQL string here } // TODO: Support persisting partitioned data source relations in Hive compatible format @@ -771,47 +771,7 @@ private[hive] case class MetastoreRelation( override protected def otherCopyArgs: Seq[AnyRef] = table :: sqlContext :: Nil - private def toHiveColumn(c: CatalogColumn): FieldSchema = { - new FieldSchema(c.name, c.dataType, c.comment.orNull) - } - - @transient val hiveQlTable: HiveTable = { - // 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.setDbName(table.database) - - val tableParameters = new java.util.HashMap[String, String]() - tTable.setParameters(tableParameters) - table.properties.foreach { case (k, v) => tableParameters.put(k, v) } - - tTable.setTableType(table.tableType match { - case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE.toString - case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE.toString - case CatalogTableType.INDEX_TABLE => HiveTableType.INDEX_TABLE.toString - case CatalogTableType.VIRTUAL_VIEW => HiveTableType.VIRTUAL_VIEW.toString - }) - - val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() - tTable.setSd(sd) - sd.setCols(table.schema.map(toHiveColumn).asJava) - tTable.setPartitionKeys(table.partitionColumns.map(toHiveColumn).asJava) - - table.storage.locationUri.foreach(sd.setLocation) - table.storage.inputFormat.foreach(sd.setInputFormat) - table.storage.outputFormat.foreach(sd.setOutputFormat) - - val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo - table.storage.serde.foreach(serdeInfo.setSerializationLib) - sd.setSerdeInfo(serdeInfo) - - val serdeParameters = new java.util.HashMap[String, String]() - table.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } - serdeInfo.setParameters(serdeParameters) - - new HiveTable(tTable) - } + @transient val hiveQlTable: HiveTable = client.toHiveTable(table) @transient override lazy val statistics: Statistics = Statistics( sizeInBytes = { @@ -845,14 +805,11 @@ private[hive] case class MetastoreRelation( } rawPartitions.map { p => - val tPartition = new org.apache.hadoop.hive.metastore.api.Partition - tPartition.setDbName(databaseName) - tPartition.setTableName(tableName) - tPartition.setValues(p.spec.values.toList.asJava) - - val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() - tPartition.setSd(sd) - sd.setCols(table.schema.map(toHiveColumn).asJava) + val hp = client.toHivePartition(p, hiveQlTable) + + // TODO: I'm not sure if the following is super necessary; + // the partitions will just get these fields from the table anyway, right? + val sd = hp.getTPartition.getSd p.storage.locationUri.foreach(sd.setLocation) p.storage.inputFormat.foreach(sd.setInputFormat) p.storage.outputFormat.foreach(sd.setOutputFormat) @@ -867,7 +824,7 @@ private[hive] case class MetastoreRelation( p.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } serdeInfo.setParameters(serdeParameters) - new Partition(hiveQlTable, tPartition) + hp } } 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 a49fe648fe59b..58010513538f6 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 @@ -380,14 +380,14 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging } else { CatalogTableType.MANAGED_TABLE }, - schema = Seq.empty[CatalogColumn], storage = CatalogStorageFormat( locationUri = None, inputFormat = None, outputFormat = None, serde = None, serdeProperties = Map.empty[String, String] - )) + ), + schema = Seq.empty[CatalogColumn]) // default storage type abbreviation (e.g. RCFile, ORC, PARQUET etc.) val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT) 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 c7527068fd5e4..95b8609cc77c1 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.client import java.io.PrintStream +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} + import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression @@ -75,7 +77,7 @@ private[hive] trait HiveClient { getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException(dbName, tableName)) } - /** Returns the metadata for the specified table or None if it doens't exist. */ + /** Returns the metadata for the specified table or None if it doesn't exist. */ def getTableOption(dbName: String, tableName: String): Option[CatalogTable] /** Creates a view with the given metadata. */ @@ -249,4 +251,12 @@ private[hive] trait HiveClient { /** Used for testing only. Removes all metadata from this instance of Hive. */ def reset(): Unit + + // --------------------------- // + // Helper conversion methods // + // --------------------------- // + + def toHiveTable(table: CatalogTable): HiveTable + + def toHivePartition(p: CatalogTablePartition, ht: HiveTable): HivePartition } 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 f89c3bce2cd9c..78a8488ac1cde 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 @@ -629,7 +629,7 @@ private[hive] class HiveClientImpl( comment = Option(hc.getComment)) } - private def toHiveTable(table: CatalogTable): HiveTable = { + override def toHiveTable(table: CatalogTable): HiveTable = { val hiveTable = new HiveTable(table.database, table.name) hiveTable.setTableType(table.tableType match { case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE @@ -663,7 +663,9 @@ private[hive] class HiveClientImpl( tbl } - private def toHivePartition(p: CatalogTablePartition, ht: HiveTable): HivePartition = { + override def toHivePartition( + p: CatalogTablePartition, + ht: HiveTable): HivePartition = { new HivePartition(ht, p.spec.asJava, p.storage.locationUri.map { l => new Path(l) }.orNull) } 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 3b3a907510134..e869c0e2bdb71 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 @@ -72,10 +72,10 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) assert(desc.storage.serdeProperties == Map((serdeConstants.SERIALIZATION_FORMAT, "\054"), (serdeConstants.FIELD_DELIM, "\054"))) - assert(desc.storage.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) - assert(desc.storage.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) assert(desc.storage.serde == - Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + Some("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) } @@ -117,9 +117,9 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { CatalogColumn("dt", "string", comment = Some("date type")) :: CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) assert(desc.storage.serdeProperties == Map()) - assert(desc.storage.inputFormat == Option("parquet.hive.DeprecatedParquetInputFormat")) - assert(desc.storage.outputFormat == Option("parquet.hive.DeprecatedParquetOutputFormat")) - assert(desc.storage.serde == Option("parquet.hive.serde.ParquetHiveSerDe")) + assert(desc.storage.inputFormat == Some("parquet.hive.DeprecatedParquetInputFormat")) + assert(desc.storage.outputFormat == Some("parquet.hive.DeprecatedParquetOutputFormat")) + assert(desc.storage.serde == Some("parquet.hive.serde.ParquetHiveSerDe")) assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) } @@ -134,9 +134,9 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(desc.schema == Seq.empty[CatalogColumn]) assert(desc.viewText == None) // TODO will be SQLText assert(desc.storage.serdeProperties == Map()) - assert(desc.storage.inputFormat == Option("org.apache.hadoop.mapred.TextInputFormat")) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.mapred.TextInputFormat")) assert(desc.storage.outputFormat == - Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) + Some("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) assert(desc.storage.serde.isEmpty) assert(desc.properties == Map()) } @@ -169,9 +169,9 @@ class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { assert(desc.schema == Seq.empty[CatalogColumn]) assert(desc.viewText == None) // TODO will be SQLText assert(desc.storage.serdeProperties == Map(("serde_p1" -> "p1"), ("serde_p2" -> "p2"))) - assert(desc.storage.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) - assert(desc.storage.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - assert(desc.storage.serde == Option("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.storage.serde == Some("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) assert(desc.properties == Map(("tbl_p1" -> "p11"), ("tbl_p2" -> "p22"))) } From 2ba19904c24951c0d06df9929c5cc3654a581344 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 18 Feb 2016 17:03:10 -0800 Subject: [PATCH 29/30] Un-ignore alter partitions test It turns out that you need to run "USE my_database" before "ALTER TABLE my_table PARTITION ..." (HIVE-2742). Geez. --- .../catalyst/catalog/InMemoryCatalog.scala | 2 + .../sql/catalyst/catalog/interface.scala | 2 + .../catalyst/catalog/CatalogTestCases.scala | 51 +++++++++++-------- .../apache/spark/sql/hive/HiveCatalog.scala | 4 ++ 4 files changed, 38 insertions(+), 21 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 72fa49f3ececc..cba4de34f2b44 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 @@ -141,6 +141,8 @@ class InMemoryCatalog extends Catalog { filterPattern(listDatabases(), pattern) } + override def setCurrentDatabase(db: String): Unit = { /* no-op */ } + // -------------------------------------------------------------------------- // Tables // -------------------------------------------------------------------------- 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 7e0071ced233f..0afad1e209e72 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 @@ -64,6 +64,8 @@ abstract class Catalog { def listDatabases(pattern: String): Seq[String] + def setCurrentDatabase(db: String): Unit + // -------------------------------------------------------------------------- // Tables // -------------------------------------------------------------------------- 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 39f09ffa78262..e0d1220d13e7c 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 @@ -407,27 +407,36 @@ abstract class CatalogTestCases extends SparkFunSuite with BeforeAndAfterEach { } } - // TODO: figure out how to fix me for Hive, which currently fails with the most helpful - // error message in the world: "Unable to alter partition. alter is not possible" - ignore("alter partitions") { - val catalog = newBasicCatalog() - val oldPart1 = catalog.getPartition("db2", "tbl2", part1.spec) - val oldPart2 = catalog.getPartition("db2", "tbl2", part2.spec) - // alter but keep spec the same - catalog.alterPartitions("db2", "tbl2", Seq( - part1.copy(storage = storageFormat.copy(locationUri = Some("usa"))), - part2.copy(storage = storageFormat.copy(locationUri = Some("china"))))) - val newPart1 = catalog.getPartition("db2", "tbl2", part1.spec) - val newPart2 = catalog.getPartition("db2", "tbl2", part2.spec) - assert(newPart1.storage.locationUri != oldPart1.storage.locationUri) - assert(newPart2.storage.locationUri != oldPart2.storage.locationUri) - assert(newPart1.storage.locationUri == Some("usa")) - assert(newPart2.storage.locationUri == Some("china")) - // 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", "tbl2", Seq(badPart1, badPart2)) + test("alter partitions") { + val catalog = newBasicCatalog() + 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." + // See HIVE-2742 for more detail. + catalog.setCurrentDatabase("db2") + val newLocation = newUriForDatabase() + // alter but keep spec the same + val oldPart1 = catalog.getPartition("db2", "tbl2", part1.spec) + val oldPart2 = catalog.getPartition("db2", "tbl2", part2.spec) + catalog.alterPartitions("db2", "tbl2", Seq( + oldPart1.copy(storage = storageFormat.copy(locationUri = Some(newLocation))), + oldPart2.copy(storage = storageFormat.copy(locationUri = Some(newLocation))))) + val newPart1 = catalog.getPartition("db2", "tbl2", part1.spec) + val newPart2 = catalog.getPartition("db2", "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", "tbl2", Seq(badPart1, badPart2)) + } + } finally { + // Remember to restore the original current database, which we assume to be "default" + catalog.setCurrentDatabase("default") } } 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 8bb6e91b49c36..21b9cfb820eaa 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 @@ -134,6 +134,10 @@ private[spark] class HiveCatalog(client: HiveClient) extends Catalog with Loggin client.listDatabases(pattern) } + override def setCurrentDatabase(db: String): Unit = withClient { + client.setCurrentDatabase(db) + } + // -------------------------------------------------------------------------- // Tables // -------------------------------------------------------------------------- From d9a77230c927a71d6ee8fbc45185379ce97e6218 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 18 Feb 2016 17:53:35 -0800 Subject: [PATCH 30/30] Fix linking error in tests This was caused by cb288da, an attempt to clean up some duplicate code. It turns out that HiveClient and HiveClientImpl cannot both refer to Hive classes due to some classloader issues. Surprise... This commit reverts part of the changes introduced in cb288da. --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 58 ++++++++++++++++--- .../spark/sql/hive/client/HiveClient.scala | 9 --- .../sql/hive/client/HiveClientImpl.scala | 4 +- 3 files changed, 53 insertions(+), 18 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 56dabe7f3bc3e..2508c314da1d4 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 @@ -771,7 +771,48 @@ private[hive] case class MetastoreRelation( override protected def otherCopyArgs: Seq[AnyRef] = table :: sqlContext :: Nil - @transient val hiveQlTable: HiveTable = client.toHiveTable(table) + private def toHiveColumn(c: CatalogColumn): FieldSchema = { + new FieldSchema(c.name, c.dataType, c.comment.orNull) + } + + // TODO: merge this with HiveClientImpl#toHiveTable + @transient val hiveQlTable: HiveTable = { + // 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.setDbName(table.database) + + val tableParameters = new java.util.HashMap[String, String]() + tTable.setParameters(tableParameters) + table.properties.foreach { case (k, v) => tableParameters.put(k, v) } + + tTable.setTableType(table.tableType match { + case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE.toString + case CatalogTableType.MANAGED_TABLE => HiveTableType.MANAGED_TABLE.toString + case CatalogTableType.INDEX_TABLE => HiveTableType.INDEX_TABLE.toString + case CatalogTableType.VIRTUAL_VIEW => HiveTableType.VIRTUAL_VIEW.toString + }) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tTable.setSd(sd) + sd.setCols(table.schema.map(toHiveColumn).asJava) + tTable.setPartitionKeys(table.partitionColumns.map(toHiveColumn).asJava) + + table.storage.locationUri.foreach(sd.setLocation) + table.storage.inputFormat.foreach(sd.setInputFormat) + table.storage.outputFormat.foreach(sd.setOutputFormat) + + val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo + table.storage.serde.foreach(serdeInfo.setSerializationLib) + sd.setSerdeInfo(serdeInfo) + + val serdeParameters = new java.util.HashMap[String, String]() + table.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + serdeInfo.setParameters(serdeParameters) + + new HiveTable(tTable) + } @transient override lazy val statistics: Statistics = Statistics( sizeInBytes = { @@ -805,11 +846,14 @@ private[hive] case class MetastoreRelation( } rawPartitions.map { p => - val hp = client.toHivePartition(p, hiveQlTable) - - // TODO: I'm not sure if the following is super necessary; - // the partitions will just get these fields from the table anyway, right? - val sd = hp.getTPartition.getSd + val tPartition = new org.apache.hadoop.hive.metastore.api.Partition + tPartition.setDbName(databaseName) + tPartition.setTableName(tableName) + tPartition.setValues(p.spec.values.toList.asJava) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tPartition.setSd(sd) + sd.setCols(table.schema.map(toHiveColumn).asJava) p.storage.locationUri.foreach(sd.setLocation) p.storage.inputFormat.foreach(sd.setInputFormat) p.storage.outputFormat.foreach(sd.setOutputFormat) @@ -824,7 +868,7 @@ private[hive] case class MetastoreRelation( p.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } serdeInfo.setParameters(serdeParameters) - hp + new Partition(hiveQlTable, tPartition) } } 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 95b8609cc77c1..6a0a089fd1f44 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 @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive.client import java.io.PrintStream -import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} - import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression @@ -252,11 +250,4 @@ private[hive] trait HiveClient { /** Used for testing only. Removes all metadata from this instance of Hive. */ def reset(): Unit - // --------------------------- // - // Helper conversion methods // - // --------------------------- // - - def toHiveTable(table: CatalogTable): HiveTable - - def toHivePartition(p: CatalogTablePartition, ht: HiveTable): HivePartition } 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 78a8488ac1cde..c7f575aad6c80 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 @@ -629,7 +629,7 @@ private[hive] class HiveClientImpl( comment = Option(hc.getComment)) } - override def toHiveTable(table: CatalogTable): HiveTable = { + private def toHiveTable(table: CatalogTable): HiveTable = { val hiveTable = new HiveTable(table.database, table.name) hiveTable.setTableType(table.tableType match { case CatalogTableType.EXTERNAL_TABLE => HiveTableType.EXTERNAL_TABLE @@ -663,7 +663,7 @@ private[hive] class HiveClientImpl( tbl } - override def toHivePartition( + private def toHivePartition( p: CatalogTablePartition, ht: HiveTable): HivePartition = { new HivePartition(ht, p.spec.asJava, p.storage.locationUri.map { l => new Path(l) }.orNull)