diff options
author | Andrew Or <andrew@databricks.com> | 2016-04-11 20:59:45 -0700 |
---|---|---|
committer | Yin Huai <yhuai@databricks.com> | 2016-04-11 20:59:45 -0700 |
commit | 83fb96403bcfb1566e9d765690744824724737ac (patch) | |
tree | 25c7e8974f72c4f4cc85086191bba87f3ade23de /sql/hive/src/main/scala/org | |
parent | e9e1adc036643c6b126237903b8e79ab379b1d32 (diff) | |
download | spark-83fb96403bcfb1566e9d765690744824724737ac.tar.gz spark-83fb96403bcfb1566e9d765690744824724737ac.tar.bz2 spark-83fb96403bcfb1566e9d765690744824724737ac.zip |
[SPARK-14132][SPARK-14133][SQL] Alter table partition DDLs
## What changes were proposed in this pull request?
This implements a few alter table partition commands using the `SessionCatalog`. In particular:
```
ALTER TABLE ... ADD PARTITION ...
ALTER TABLE ... DROP PARTITION ...
ALTER TABLE ... RENAME PARTITION ... TO ...
```
The following operations are not supported, and an `AnalysisException` with a helpful error message will be thrown if the user tries to use them:
```
ALTER TABLE ... EXCHANGE PARTITION ...
ALTER TABLE ... ARCHIVE PARTITION ...
ALTER TABLE ... UNARCHIVE PARTITION ...
ALTER TABLE ... TOUCH ...
ALTER TABLE ... COMPACT ...
ALTER TABLE ... CONCATENATE
MSCK REPAIR TABLE ...
```
## How was this patch tested?
`DDLSuite`, `DDLCommandSuite` and `HiveDDLCommandSuite`
Author: Andrew Or <andrew@databricks.com>
Closes #12220 from andrewor14/alter-partition-ddl.
Diffstat (limited to 'sql/hive/src/main/scala/org')
3 files changed, 23 insertions, 29 deletions
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index a49ce33ba1..482f47428d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -219,26 +219,7 @@ private[spark] class HiveExternalCatalog(client: HiveClient) extends ExternalCat parts: Seq[TablePartitionSpec], 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) - } + client.dropPartitions(db, table, parts, ignoreIfNotExists) } override def renamePartitions( 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 94794b1572..6f7e7bf451 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 @@ -120,16 +120,13 @@ private[hive] trait HiveClient { ignoreIfExists: Boolean): Unit /** - * 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. + * Drop one or many partitions in the given table, assuming they exist. */ def dropPartitions( db: String, table: String, - specs: Seq[ExternalCatalog.TablePartitionSpec]): Unit + specs: Seq[ExternalCatalog.TablePartitionSpec], + ignoreIfNotExists: Boolean): Unit /** * Rename one or many existing table partitions, assuming they 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 a037671ef0..39e26acd7f 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 org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.cli.CliSessionState 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, Function => HiveFunction, FunctionType, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} @@ -367,9 +367,25 @@ private[hive] class HiveClientImpl( override def dropPartitions( db: String, table: String, - specs: Seq[ExternalCatalog.TablePartitionSpec]): Unit = withHiveState { + specs: Seq[ExternalCatalog.TablePartitionSpec], + ignoreIfNotExists: Boolean): Unit = withHiveState { // TODO: figure out how to drop multiple partitions in one call - specs.foreach { s => client.dropPartition(db, table, s.values.toList.asJava, true) } + val hiveTable = client.getTable(db, table, true /* throw exception */) + specs.foreach { s => + // The provided spec here can be a partial spec, i.e. it will match all partitions + // whose specs are supersets of this partial spec. E.g. If a table has partitions + // (b='1', c='1') and (b='1', c='2'), a partial spec of (b='1') will match both. + val matchingParts = client.getPartitions(hiveTable, s.asJava).asScala + if (matchingParts.isEmpty && !ignoreIfNotExists) { + throw new AnalysisException( + s"partition to drop '$s' does not exist in table '$table' database '$db'") + } + matchingParts.foreach { hivePartition => + val dropOptions = new PartitionDropOptions + dropOptions.ifExists = ignoreIfNotExists + client.dropPartition(db, table, hivePartition.getValues, dropOptions) + } + } } override def renamePartitions( |