Skip to content

Commit a23debd

Browse files
gatorsmilecloud-fan
authored andcommitted
[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 <[email protected]> Closes #16583 from gatorsmile/disallowEmptyPartColValue.
1 parent a83accf commit a23debd

File tree

5 files changed

+106
-8
lines changed

5 files changed

+106
-8
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 24 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -331,7 +331,7 @@ class SessionCatalog(
331331
def loadPartition(
332332
name: TableIdentifier,
333333
loadPath: String,
334-
partition: TablePartitionSpec,
334+
spec: TablePartitionSpec,
335335
isOverwrite: Boolean,
336336
holdDDLTime: Boolean,
337337
inheritTableSpecs: Boolean,
@@ -340,8 +340,9 @@ class SessionCatalog(
340340
val table = formatTableName(name.table)
341341
requireDbExists(db)
342342
requireTableExists(TableIdentifier(table, Some(db)))
343+
requireNonEmptyValueInPartitionSpec(Seq(spec))
343344
externalCatalog.loadPartition(
344-
db, table, loadPath, partition, isOverwrite, holdDDLTime, inheritTableSpecs, isSrcLocal)
345+
db, table, loadPath, spec, isOverwrite, holdDDLTime, inheritTableSpecs, isSrcLocal)
345346
}
346347

347348
def defaultTablePath(tableIdent: TableIdentifier): String = {
@@ -693,6 +694,7 @@ class SessionCatalog(
693694
requireDbExists(db)
694695
requireTableExists(TableIdentifier(table, Option(db)))
695696
requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName))
697+
requireNonEmptyValueInPartitionSpec(parts.map(_.spec))
696698
externalCatalog.createPartitions(db, table, parts, ignoreIfExists)
697699
}
698700

@@ -711,6 +713,7 @@ class SessionCatalog(
711713
requireDbExists(db)
712714
requireTableExists(TableIdentifier(table, Option(db)))
713715
requirePartialMatchedPartitionSpec(specs, getTableMetadata(tableName))
716+
requireNonEmptyValueInPartitionSpec(specs)
714717
externalCatalog.dropPartitions(db, table, specs, ignoreIfNotExists, purge, retainData)
715718
}
716719

@@ -731,6 +734,8 @@ class SessionCatalog(
731734
requireTableExists(TableIdentifier(table, Option(db)))
732735
requireExactMatchedPartitionSpec(specs, tableMetadata)
733736
requireExactMatchedPartitionSpec(newSpecs, tableMetadata)
737+
requireNonEmptyValueInPartitionSpec(specs)
738+
requireNonEmptyValueInPartitionSpec(newSpecs)
734739
externalCatalog.renamePartitions(db, table, specs, newSpecs)
735740
}
736741

@@ -749,6 +754,7 @@ class SessionCatalog(
749754
requireDbExists(db)
750755
requireTableExists(TableIdentifier(table, Option(db)))
751756
requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName))
757+
requireNonEmptyValueInPartitionSpec(parts.map(_.spec))
752758
externalCatalog.alterPartitions(db, table, parts)
753759
}
754760

@@ -762,6 +768,7 @@ class SessionCatalog(
762768
requireDbExists(db)
763769
requireTableExists(TableIdentifier(table, Option(db)))
764770
requireExactMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName))
771+
requireNonEmptyValueInPartitionSpec(Seq(spec))
765772
externalCatalog.getPartition(db, table, spec)
766773
}
767774

@@ -781,6 +788,7 @@ class SessionCatalog(
781788
requireTableExists(TableIdentifier(table, Option(db)))
782789
partialSpec.foreach { spec =>
783790
requirePartialMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName))
791+
requireNonEmptyValueInPartitionSpec(Seq(spec))
784792
}
785793
externalCatalog.listPartitionNames(db, table, partialSpec)
786794
}
@@ -801,6 +809,7 @@ class SessionCatalog(
801809
requireTableExists(TableIdentifier(table, Option(db)))
802810
partialSpec.foreach { spec =>
803811
requirePartialMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName))
812+
requireNonEmptyValueInPartitionSpec(Seq(spec))
804813
}
805814
externalCatalog.listPartitions(db, table, partialSpec)
806815
}
@@ -819,6 +828,19 @@ class SessionCatalog(
819828
externalCatalog.listPartitionsByFilter(db, table, predicates)
820829
}
821830

831+
/**
832+
* Verify if the input partition spec has any empty value.
833+
*/
834+
private def requireNonEmptyValueInPartitionSpec(specs: Seq[TablePartitionSpec]): Unit = {
835+
specs.foreach { s =>
836+
if (s.values.exists(_.isEmpty)) {
837+
val spec = s.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]")
838+
throw new AnalysisException(
839+
s"Partition spec is invalid. The spec ($spec) contains an empty partition column value")
840+
}
841+
}
842+
}
843+
822844
/**
823845
* Verify if the input partition spec exactly matches the existing defined partition spec
824846
* The columns must be the same but the orders could be different.

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -848,6 +848,8 @@ abstract class CatalogTestUtils {
848848
CatalogTablePartition(Map("a" -> "5", "b" -> "6", "c" -> "7"), storageFormat)
849849
lazy val partWithUnknownColumns =
850850
CatalogTablePartition(Map("a" -> "5", "unknown" -> "6"), storageFormat)
851+
lazy val partWithEmptyValue =
852+
CatalogTablePartition(Map("a" -> "3", "b" -> ""), storageFormat)
851853
lazy val funcClass = "org.apache.spark.myFunc"
852854

853855
/**

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala

Lines changed: 65 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -625,6 +625,13 @@ class SessionCatalogSuite extends PlanTest {
625625
}
626626
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " +
627627
"the partition spec (a, b) defined in table '`db2`.`tbl2`'"))
628+
e = intercept[AnalysisException] {
629+
catalog.createPartitions(
630+
TableIdentifier("tbl2", Some("db2")),
631+
Seq(partWithEmptyValue, part1), ignoreIfExists = true)
632+
}
633+
assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " +
634+
"empty partition column value"))
628635
}
629636

630637
test("drop partitions") {
@@ -722,6 +729,16 @@ class SessionCatalogSuite extends PlanTest {
722729
assert(e.getMessage.contains(
723730
"Partition spec is invalid. The spec (a, unknown) must be contained within " +
724731
"the partition spec (a, b) defined in table '`db2`.`tbl2`'"))
732+
e = intercept[AnalysisException] {
733+
catalog.dropPartitions(
734+
TableIdentifier("tbl2", Some("db2")),
735+
Seq(partWithEmptyValue.spec, part1.spec),
736+
ignoreIfNotExists = false,
737+
purge = false,
738+
retainData = false)
739+
}
740+
assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " +
741+
"empty partition column value"))
725742
}
726743

727744
test("get partition") {
@@ -767,6 +784,11 @@ class SessionCatalogSuite extends PlanTest {
767784
}
768785
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " +
769786
"the partition spec (a, b) defined in table '`db2`.`tbl1`'"))
787+
e = intercept[AnalysisException] {
788+
catalog.getPartition(TableIdentifier("tbl1", Some("db2")), partWithEmptyValue.spec)
789+
}
790+
assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " +
791+
"empty partition column value"))
770792
}
771793

772794
test("rename partitions") {
@@ -834,6 +856,13 @@ class SessionCatalogSuite extends PlanTest {
834856
}
835857
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " +
836858
"the partition spec (a, b) defined in table '`db2`.`tbl1`'"))
859+
e = intercept[AnalysisException] {
860+
catalog.renamePartitions(
861+
TableIdentifier("tbl1", Some("db2")),
862+
Seq(part1.spec), Seq(partWithEmptyValue.spec))
863+
}
864+
assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " +
865+
"empty partition column value"))
837866
}
838867

839868
test("alter partitions") {
@@ -893,6 +922,11 @@ class SessionCatalogSuite extends PlanTest {
893922
}
894923
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " +
895924
"the partition spec (a, b) defined in table '`db2`.`tbl1`'"))
925+
e = intercept[AnalysisException] {
926+
catalog.alterPartitions(TableIdentifier("tbl1", Some("db2")), Seq(partWithEmptyValue))
927+
}
928+
assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " +
929+
"empty partition column value"))
896930
}
897931

898932
test("list partition names") {
@@ -914,10 +948,24 @@ class SessionCatalogSuite extends PlanTest {
914948

915949
test("list partition names with invalid partial partition spec") {
916950
val catalog = new SessionCatalog(newBasicCatalog())
917-
intercept[AnalysisException] {
951+
var e = intercept[AnalysisException] {
952+
catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")),
953+
Some(partWithMoreColumns.spec))
954+
}
955+
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, b, c) must be " +
956+
"contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'"))
957+
e = intercept[AnalysisException] {
958+
catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")),
959+
Some(partWithUnknownColumns.spec))
960+
}
961+
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must be " +
962+
"contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'"))
963+
e = intercept[AnalysisException] {
918964
catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")),
919-
Some(Map("unknown" -> "unknown")))
965+
Some(partWithEmptyValue.spec))
920966
}
967+
assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " +
968+
"empty partition column value"))
921969
}
922970

923971
test("list partitions") {
@@ -937,10 +985,22 @@ class SessionCatalogSuite extends PlanTest {
937985

938986
test("list partitions with invalid partial partition spec") {
939987
val catalog = new SessionCatalog(newBasicCatalog())
940-
intercept[AnalysisException] {
941-
catalog.listPartitions(
942-
TableIdentifier("tbl2", Some("db2")), Some(Map("unknown" -> "unknown")))
988+
var e = intercept[AnalysisException] {
989+
catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(partWithMoreColumns.spec))
990+
}
991+
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, b, c) must be " +
992+
"contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'"))
993+
e = intercept[AnalysisException] {
994+
catalog.listPartitions(TableIdentifier("tbl2", Some("db2")),
995+
Some(partWithUnknownColumns.spec))
996+
}
997+
assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must be " +
998+
"contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'"))
999+
e = intercept[AnalysisException] {
1000+
catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(partWithEmptyValue.spec))
9431001
}
1002+
assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " +
1003+
"empty partition column value"))
9441004
}
9451005

9461006
test("list partitions when database/table does not exist") {

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -471,6 +471,7 @@ private[hive] class HiveClientImpl(
471471
// do the check at first and collect all the matching partitions
472472
val matchingParts =
473473
specs.flatMap { s =>
474+
assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
474475
// The provided spec here can be a partial spec, i.e. it will match all partitions
475476
// whose specs are supersets of this partial spec. E.g. If a table has partitions
476477
// (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(
545546
// -1 for result limit means "no limit/return all"
546547
client.getPartitionNames(table.database, table.identifier.table, -1)
547548
case Some(s) =>
549+
assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
548550
client.getPartitionNames(table.database, table.identifier.table, s.asJava, -1)
549551
}
550552
hivePartitionNames.asScala.sorted
@@ -568,7 +570,9 @@ private[hive] class HiveClientImpl(
568570
val hiveTable = toHiveTable(table)
569571
val parts = spec match {
570572
case None => shim.getAllPartitions(client, hiveTable).map(fromHivePartition)
571-
case Some(s) => client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition)
573+
case Some(s) =>
574+
assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
575+
client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition)
572576
}
573577
HiveCatalogMetrics.incrementFetchedPartitions(parts.length)
574578
parts

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -247,6 +247,16 @@ class HiveDDLSuite
247247
}
248248
}
249249

250+
test("SPARK-19129: drop partition with a empty string will drop the whole table") {
251+
val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name")
252+
df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable")
253+
val e = intercept[AnalysisException] {
254+
spark.sql("alter table partitionedTable drop partition(partCol1='')")
255+
}.getMessage
256+
assert(e.contains("Partition spec is invalid. The spec ([partCol1=]) contains an empty " +
257+
"partition column value"))
258+
}
259+
250260
test("add/drop partitions - external table") {
251261
val catalog = spark.sessionState.catalog
252262
withTempDir { tmpDir =>

0 commit comments

Comments
 (0)