aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authorgatorsmile <gatorsmile@gmail.com>2017-01-18 02:01:30 +0800
committerWenchen Fan <wenchen@databricks.com>2017-01-18 02:01:30 +0800
commita23debd7bc8f85ea49c54b8cf3cd112cf0a803ff (patch)
treeabb2b167618351b3181ed4956dd93daf356c7359 /sql/hive
parenta83accfcfd6a92afac5040c50577258ab83d10dd (diff)
downloadspark-a23debd7bc8f85ea49c54b8cf3cd112cf0a803ff.tar.gz
spark-a23debd7bc8f85ea49c54b8cf3cd112cf0a803ff.tar.bz2
spark-a23debd7bc8f85ea49c54b8cf3cd112cf0a803ff.zip
[SPARK-19129][SQL] SessionCatalog: Disallow empty part col values in partition spec
### What changes were proposed in this pull request? Empty partition column values are not valid for partition specification. Before this PR, we accept users to do it; however, Hive metastore does not detect and disallow it too. Thus, users hit the following strange error. ```Scala val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name") df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable") spark.sql("alter table partitionedTable drop partition(partCol1='')") spark.table("partitionedTable").show() ``` In the above example, the WHOLE table is DROPPED when users specify a partition spec containing only one partition column with empty values. When the partition columns contains more than one, Hive metastore APIs simply ignore the columns with empty values and treat it as partial spec. This is also not expected. This does not follow the actual Hive behaviors. This PR is to disallow users to specify such an invalid partition spec in the `SessionCatalog` APIs. ### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #16583 from gatorsmile/disallowEmptyPartColValue.
Diffstat (limited to 'sql/hive')
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala6
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala10
2 files changed, 15 insertions, 1 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 5c0e2f6ec4..9a6144c5e3 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
@@ -471,6 +471,7 @@ private[hive] class HiveClientImpl(
// do the check at first and collect all the matching partitions
val matchingParts =
specs.flatMap { s =>
+ assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
// 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.
@@ -545,6 +546,7 @@ private[hive] class HiveClientImpl(
// -1 for result limit means "no limit/return all"
client.getPartitionNames(table.database, table.identifier.table, -1)
case Some(s) =>
+ assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
client.getPartitionNames(table.database, table.identifier.table, s.asJava, -1)
}
hivePartitionNames.asScala.sorted
@@ -568,7 +570,9 @@ private[hive] class HiveClientImpl(
val hiveTable = toHiveTable(table)
val parts = spec match {
case None => shim.getAllPartitions(client, hiveTable).map(fromHivePartition)
- case Some(s) => client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition)
+ case Some(s) =>
+ assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
+ client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition)
}
HiveCatalogMetrics.incrementFetchedPartitions(parts.length)
parts
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index e3f1667249..ef62be39cd 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -247,6 +247,16 @@ class HiveDDLSuite
}
}
+ test("SPARK-19129: drop partition with a empty string will drop the whole table") {
+ val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name")
+ df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable")
+ val e = intercept[AnalysisException] {
+ spark.sql("alter table partitionedTable drop partition(partCol1='')")
+ }.getMessage
+ assert(e.contains("Partition spec is invalid. The spec ([partCol1=]) contains an empty " +
+ "partition column value"))
+ }
+
test("add/drop partitions - external table") {
val catalog = spark.sessionState.catalog
withTempDir { tmpDir =>