-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17612][SQL] Support DESCRIBE table PARTITION SQL syntax
#15168
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
Conversation
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.
Can you add some validation for the output returned.
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.
Hi, @skambha .
Thank you for review. BTW, what kind of validation do you want? If we have a DESC command output validation test cases, it will be the same.
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.
I was thinking - of verifying that the output result points out the partition columns correctly.
Is there a testcase that covers this already? If so that is fine.
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.
Actually, as you see in the PR description, the output result is the same with the DESC command with PARTITION spec. In other words, the output does not depend on the given spec. Only, this command raises exceptions if the given partition does not exists.
the output result points out the partition columns correctly.
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 supports describe based on the partition specified and will list the details for the particular partition when used with formatted or extended option.
DESCRIBE formatted part_table partition (d='abc')
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Describe
@dongjoon-hyun, this might be beyond the scope of this PR but this would be useful if there are a lot of partitions and we want to find details for a given partition. What do you think? Thanks.
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.
@skambha . Let's wait some feedback from committers. Before putting more effort on this, I want to know if this kind of PR about DESC regression will be merged into master.
|
Test build #65680 has finished for PR 15168 at commit
|
|
Hi, @rxin . |
|
Hi, @hvanhovell . |
hvanhovell
left a comment
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.
@dongjoon-hyun am I right to assume that this currently does not describe the partition the user requests? Why not implement the command now you are at it?
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.
Could you collect here and add some names? It is a bit cryptic at the moment?
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.
Does this touch the result? How is the partition information added to the result?
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.
It is possible to test the output with SQLQueryTestSuite? Or does that have portability issues?
|
Thank you, @hvanhovell . Yes, currently, as you mentioned, Like Spark 1.6, I'll add that, too. |
|
Hi, @hvanhovell . |
|
Hi, @hvanhovell . I updated the followings.
Could you review this again when you have some time? |
|
I also like this feature. The Hive-generated statistics for partitioned tables are stored in partition-level properties. See the PR: #15158 We need it before https://issues.apache.org/jira/browse/SPARK-17129 is done; otherwise, users are unable to read the statistics of partitioned tables. BTW, will try to review this PR tonight. |
|
Thank you, @gatorsmile . |
|
Test build #65842 has finished for PR 15168 at commit
|
|
Test build #65841 has finished for PR 15168 at commit
|
|
Oh, my bad. I'll move back the testcase into the original test suite. |
|
Test build #65859 has finished for PR 15168 at commit
|
|
The failure seems to be irrelevant. Retest this please. |
|
retest this please |
|
Test build #65861 has finished for PR 15168 at commit
|
|
The implementation of this PR does not support data source tables. Please try the following example. spark
.range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write
.bucketBy(2, "b").sortBy("c").partitionBy("d")
.saveAsTable("t1")
sql("DESC FORMATTED t1 partition (d=1)").show(100, false)The error you will get is: |
|
Thank you, @gatorsmile . I'll fix that soon. |
|
Hi, @gatorsmile . It's the same behavior with Apache Spark 1.6.2. This PR doesn't make any regression for Datasource tables. Actually, the error you mentioned is raised by scala> sc.version
res5: String = 1.6.2
scala> sqlContext.range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write.partitionBy("d").saveAsTable("t1")
16/09/24 08:24:51 WARN HiveContext$$anon$2: Persisting partitioned data source relation `t1` into Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. Input path(s):
file:/user/hive/warehouse/t1
scala> sql("DESC FORMATTED t1 partition (d=1)").show(100, false)
16/09/24 08:25:06 ERROR DDLTask: java.lang.RuntimeException: cannot find field null from [0:col]
at org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.getStandardStructFieldRef(ObjectInspectorUtils.java:416) |
|
Yeah, I knew. : ) This PR is to support |
|
Actually, I think there was a reason not to do that in Spark 1.6.2. In Spark 2.0, is there any new way to lookup the partition info from Parquet Datasource through Anyway, if we need to support that here. This PR needs to be IMPROVEMENT jira issue. So far, it was BUG issue to recover the regression. :) |
|
Yeah, please change the JIRA to improvement. I did not check the code of Spark 1.6.2, but, since Spark 2.0, we provide a native support for |
|
Sure. I'll investigate from that. Thank you for the pointer, @gatorsmile . |
|
Partitioned View is another potential issue for this PR. FYI, just submitted a related PR #15233 |
|
Hi, @gatorsmile . I investigated the related PRs.
Unfortunately, those are extracting the partition column names from table properties. But, table properties does not have partition column values. Currently, Spark does not hold the partitioned column values of In this case, I'd like to make another issue for that since that's really a new feature. Since it's not supported Spark 1.6, I think it's nice-to-have item, not a must-to-have item in this PR. How do you think about that? |
|
Thank you for your investigation! How about generating a user-friendly error message in these unsupported cases? The error messages from Hive is very confusing to the external users. |
|
Yep. That's would be great. Thank you for guidance always! |
|
Test build #65877 has finished for PR 15168 at commit
|
|
Thank you for review @gatorsmile . Hi, @hvanhovell . |
|
@dongjoon-hyun sorry about that. Is the PR description (including all the exceptions) up to date? |
|
Never mind~. Thank you for coming back! I'll check the PR description again and make up-to-date. |
hvanhovell
left a comment
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.
Looks pretty good. Left one comment.
| val (valid, nonValid) = visitPartitionSpec(ctx.partitionSpec).partition(_._2.isDefined) | ||
| if (nonValid.nonEmpty) { | ||
| // For non-valid specification for `DESC` command, this raises a ParseException. | ||
| return null |
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.
Just throw a meaningful parse exception and simplify this logic.
visitPartitionSpec(ctx.partitionSpec).map {
case (key, Some(value)) => key -> value
case _ => throw new ParseException("...", ctx)
}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.
Thank you. I'll fix soon.
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.
Done. Now, Jenkins is running.
|
Now, the PR description is also up-to-date. |
|
Test build #65988 has finished for PR 15168 at commit
|
|
Test build #65989 has finished for PR 15168 at commit
|
| @@ -0,0 +1,24 @@ | |||
| CREATE TABLE t (a STRING, b INT) PARTITIONED BY (c STRING, d STRING); | |||
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.
Drop the table?
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.
Done. Thank you!
|
Test build #65993 has finished for PR 15168 at commit
|
|
Test build #66001 has finished for PR 15168 at commit
|
|
Finally, it passed! @hvanhovell . |
|
Hi, @hvanhovell . |
|
Hi, @hvanhovell . I thought it's a regression bug fix. |
|
@dongjoon-hyun we made the decision to regress here willingly. So I think it is a new feature, also this is currently not backportable. LGTM. Merging to master. Thanks. |
|
Oh, I see. Does that mean it's not allowed to be backported due to the policy although I made a patch for branch-2.0? |
|
Thank you so much, @hvanhovell , @gatorsmile , and @skambha ! |
|
When implementing the native DDL support, we blocked a few DDL statements due to the resource limit. Thus, this PR is not a bug fix. In theory, we should not add new features into the previous releases. |
|
@dongjoon-hyun the risk of this PR breaking something is quite low; so I am open to backporting as long as long as it does not create too much issues. BTW: did you open a backport? this was against master? |
|
@gatorsmile , @hvanhovell . Yep. I knew the policy about new feature issues and bug fix issues. So, I wish this PR is considered as bug fix issue from the beginning. Ya, it's a new feature indeed. @hvanhovell . I didn't make that yet since this PR wasn't merged before. I'll try to make a backport PR for that to receive a chance to be reviewed. That will be mostly the same code. If you think that's too much at that time, I'll close the backport PR then. Thank you again, All. |
What changes were proposed in this pull request?
This PR implements
DESCRIBE table PARTITIONSQL Syntax again. It was supported until Spark 1.6.2, but was dropped since 2.0.0.Spark 1.6.2
Spark 2.0
How was this patch tested?
Pass the Jenkins tests with a new testcase.