-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-32508][SQL] Disallow empty part col values in partition spec before static partition writing #29316
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-32508][SQL] Disallow empty part col values in partition spec before static partition writing #29316
Changes from all commits
224b979
221564b
e012e7a
ab4e04c
232a835
a7b9a17
65f781a
965ed5a
b969c73
c8ac369
76acc07
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -847,4 +847,26 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-32508 " + | ||
| "Disallow empty part col values in partition spec before static partition writing") { | ||
| withTable("t1") { | ||
| spark.sql( | ||
| """ | ||
| |CREATE TABLE t1 (c1 int) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why is this a problem only for hive tables?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
In the case that Perhaps this check can only be performed when
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hive calls public Partition getPartition(...){
|| (val != null && val.length() == 0)) {
throw new HiveException("get partition: Value for key "
+ field.getName() + " is null or empty");
}
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
what's the behavior then?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
When writing static partition or dynamic partition, the When
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what's the behavior of hive? Does hive treat
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hive> INSERT OVERWRITE TABLE t1 PARTITION(d='') select 1; hive> INSERT OVERWRITE TABLE t1 PARTITION(d) select 1,'' as d; |
||
| |PARTITIONED BY (d string) | ||
| """.stripMargin) | ||
|
|
||
| val e = intercept[AnalysisException] { | ||
| spark.sql( | ||
| """ | ||
| |INSERT OVERWRITE TABLE t1 PARTITION(d='') | ||
| |SELECT 1 | ||
| """.stripMargin) | ||
| }.getMessage | ||
|
|
||
| assert(!e.contains("get partition: Value for key d is null or empty")) | ||
| assert(e.contains("Partition spec 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.
So the partition column can be empty string if it's dynamic. Shall we convert the empty string/null in partition spec to
__HIVE_DEFAULT_PARTITION__before callinglistPartitions/loadPartition?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.
Generally speaking, it is meaningless for the partition value to be empty, so the static partition value is not allowed to be empty.
Dynamic partition may be that the user does not know that the partition field is null or empty, and finally wrote the
__HIVE_DEFAULT_PARTITION__partition.listPartitionsThe partition value the user sees is
__HIVE_DEFAULT_PARTITION__, so the user will not specify the partition value empty to query the partition details.loadPartitionBecause in
DynamicPartitionDataWriter#partitionPathExpression, the partition value will be null or emtpy converted to__HIVE_DEFAULT_PARTITION__, so it can be executed successfully without the need to increase early conversion.