-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-19129] [SQL] SessionCatalog: Disallow empty part col values in partition spec #16583
Conversation
Test build #71358 has finished for PR 16583 at commit
|
retest this please |
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='')") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what's the behavior of hive? also throw exception?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hive (v2.1.1) does not throw exception / error message here.
ALTER TABLE partitioned_table DROP PARTITION(ds = '') ;
OK
Time taken: 0.152 seconds
Given that (creating / inserting / querying) partitions with empty string is not allowed, DROP PARTITIONS going through seems inconsistent behavior to me. It might have made sense for supporting regexes but as per Hive language specification, partition spec has to be a plain string. If there is no way to create partitions with empty partition column name, allowing DROP seems werid. +1 for throwing exception ..... unless the general consensus about hive compatibility is to be exact same behavior (including such weirdness).
INSERT OVERWRITE TABLE partitioned_table PARTITION(ds = '') SELECT key AS user_id, value AS name FROM src;
FAILED: SemanticException [Error 10006]: Line 1:49 Partition not found ''''
ALTER TABLE partitioned_table ADD PARTITION(ds = '') ;
FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. partition spec is invalid; field ds does not exist or is empty
DESC FORMATTED partitioned_table PARTITION(ds = '') ;
FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. cannot find field null from [0:user_id, 1:name]
TRUNCATE TABLE partitioned_table PARTITION(ds = '') ;
FAILED: SemanticException [Error 10006]: Partition not found {ds=}
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@tejasapatil Thank you for your research
So far, we are not completely following Hive in the partition-related DDL commands. DROP PARTITION
is an example. If the users-specified spec does not exist, we will throw an exception. Instead, Hive just silently ignores it without any exception, but Hive will always report which partition is dropped after the command. Thus, maybe we can improve this in the future PR.
Thus, this PR is to follow the same way to block the invalid inputs. That is, throwing an exception when the input partition spec is not valid.
Test build #71362 has finished for PR 16583 at commit
|
@@ -937,10 +985,22 @@ class SessionCatalogSuite extends PlanTest { | |||
|
|||
test("list partitions with invalid partial partition spec") { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we merge this test with https://github.com/apache/spark/pull/16583/files#diff-68b981fa0a91ef20dc032d93ad0fdc52R949?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The above one is verifying the catalog.listPartitionNames
and this one is verifying catalog.listPartitions
. Should we keep them separate?
@@ -568,7 +569,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") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we also add the assert in getPartitionNames
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, it has the same issue.
Test build #71479 has started for PR 16583 at commit |
retest this please |
Test build #71505 has finished for PR 16583 at commit
|
LGTM, merging to master/2.1! |
…partition spec 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. Added test cases Author: gatorsmile <[email protected]> Closes #16583 from gatorsmile/disallowEmptyPartColValue. (cherry picked from commit a23debd) Signed-off-by: Wenchen Fan <[email protected]>
…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 apache#16583 from gatorsmile/disallowEmptyPartColValue.
…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 apache#16583 from gatorsmile/disallowEmptyPartColValue.
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.
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