-
Notifications
You must be signed in to change notification settings - Fork 3k
Spark: Add support for describing/showing views #9513
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
9d20e42 to
89aa0e7
Compare
89aa0e7 to
da61115
Compare
.../main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala
Outdated
Show resolved
Hide resolved
| } | ||
|
|
||
| private def showColumns(view: View, builder: StringBuilder): Unit = { | ||
| val columns = view.schema().fields |
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.
This should only be produced if it was part of the original CREATE VIEW command. I think we probably need to store these somewhere in ViewVersion. In addition, if they are set then the schema should be validated against them.
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 ran into this case when I was implementing this so I compared this with how SHOW CREATE TABLE <tbl> does it. Here it can be seen that it just takes the latest schema of the table and uses it for the output. So should we be aligning SHOW CREATE VIEW with SHOW CREATE TABLE here?
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 think this is fine for now, but we should consider how to handle the aliases in the future
...k-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
Outdated
Show resolved
Hide resolved
aa786db to
c3b1165
Compare
| replace = replace) | ||
|
|
||
| case ShowViews(UnresolvedNamespace(Seq()), pattern, output) if isViewCatalog(catalogManager.currentCatalog) => | ||
| ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, Seq.empty), pattern, output) |
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.
Won't this case be handled the same way by running CatalogAndNamespace.unapply(Seq()) in the next rule?
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.
unfortunately no, as CatalogAndNamespace requires the namespace to be non-empty as can be seen in https://github.com/apache/spark/blob/abaec2804fb48d2d378f7c3e99733af97283fa22/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala#L84. Therefore we need to handle the case with an empty namespace separately here (the same is done in Spark)
.../main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala
Outdated
Show resolved
Hide resolved
spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
Outdated
Show resolved
Hide resolved
...nsions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala
Outdated
Show resolved
Hide resolved
| row(NAMESPACE.toString(), "prefixV2", false), | ||
| row(NAMESPACE.toString(), "prefixV3", false), | ||
| row(NAMESPACE.toString(), "v1", false), | ||
| tempView); |
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 trust you that this is Spark's behavior. But this is weird because it isn't in the catalog.
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.
Also weird when filtering by NAMESPACE below.
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.
Yes this is the expected Spark behavior as can be seen in https://spark.apache.org/docs/3.5.0/sql-ref-syntax-aux-show-views.html. Spark always lists temp views (which don't belong to any catalog)
spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
Outdated
Show resolved
Hide resolved
spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
Outdated
Show resolved
Hide resolved
|
I pointed out a few things that I think I would change, but there is nothing blocking this. +1 overall. Please fix what you agree with (including the |
This adds support for: * `DESCRIBE <viewName>` / `DESCRIBE EXTENDED <viewName>` * `SHOW VIEWS` / `SHOW VIEWS LIKE <pattern>` * `SHOW TBLPROPERTIES <viewName>` * `SHOW CREATE TABLE <viewName>`
c3b1165 to
8a6063f
Compare
|
thank for reviewing @rdblue, I addressed everything and will go ahead and merge this |
This adds support for: * `DESCRIBE <viewName>` / `DESCRIBE EXTENDED <viewName>` * `SHOW VIEWS` / `SHOW VIEWS LIKE <pattern>` * `SHOW TBLPROPERTIES <viewName>` * `SHOW CREATE TABLE <viewName>`
This adds support for:
DESCRIBE <viewName>/DESCRIBE EXTENDED <viewName>SHOW VIEWS/SHOW VIEWS LIKE <pattern>SHOW TBLPROPERTIES <viewName>SHOW CREATE TABLE <viewName>