This is an automated email from the ASF dual-hosted git repository.
szehon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git
The following commit(s) were added to refs/heads/main by this push:
new 59e937761a Spark 3.4, 3.5: SHOW VIEWS failed with AssertionError
(#10442)
59e937761a is described below
commit 59e937761aac2dddf14d331268aeb0b9adec6908
Author: Huaxin Gao <[email protected]>
AuthorDate: Wed Jun 5 10:31:49 2024 -0700
Spark 3.4, 3.5: SHOW VIEWS failed with AssertionError (#10442)
---
.../spark/sql/catalyst/analysis/RewriteViewCommands.scala | 11 +++++++----
.../java/org/apache/iceberg/spark/extensions/TestViews.java | 5 +++++
.../integration/java/org/apache/iceberg/spark/SmokeTest.java | 9 +++++++++
.../spark/sql/catalyst/analysis/RewriteViewCommands.scala | 11 +++++++----
.../java/org/apache/iceberg/spark/extensions/TestViews.java | 5 +++++
.../integration/java/org/apache/iceberg/spark/SmokeTest.java | 9 +++++++++
6 files changed, 42 insertions(+), 8 deletions(-)
diff --git
a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
index 377025f926..3e854dfcb9 100644
---
a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
+++
b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
@@ -66,10 +66,13 @@ case class RewriteViewCommands(spark: SparkSession) extends
Rule[LogicalPlan] wi
allowExisting = allowExisting,
replace = replace)
- case ShowViews(UnresolvedNamespace(Seq()), pattern, output)
- if ViewUtil.isViewCatalog(catalogManager.currentCatalog) =>
- ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog,
catalogManager.currentNamespace),
- pattern, output)
+ case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) =>
+ if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) {
+ ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog,
catalogManager.currentNamespace),
+ pattern, output)
+ } else {
+ view
+ }
case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)),
pattern, output)
if ViewUtil.isViewCatalog(catalog) =>
diff --git
a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
index 624b4e3549..65b8669c35 100644
---
a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
+++
b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
@@ -1463,6 +1463,11 @@ public class TestViews extends SparkExtensionsTestBase {
.contains(
// spark stores temp views case-insensitive by default
row("global_temp", "globalviewforlisting", true), tempView);
+
+ sql("USE spark_catalog");
+ assertThat(sql("SHOW VIEWS")).contains(tempView);
+
+ assertThat(sql("SHOW VIEWS IN default")).contains(tempView);
}
@Test
diff --git
a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
index 25d7e74715..59bc70854c 100644
---
a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
+++
b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
@@ -18,6 +18,8 @@
*/
package org.apache.iceberg.spark;
+import static org.assertj.core.api.Assertions.assertThat;
+
import java.io.IOException;
import java.util.Map;
import org.apache.iceberg.Table;
@@ -165,6 +167,13 @@ public class SmokeTest extends SparkExtensionsTestBase {
Assert.assertEquals("Should be partitioned on 3 columns", 3,
third.spec().fields().size());
}
+ @Test
+ public void showView() {
+ sql("DROP VIEW IF EXISTS %s", "test");
+ sql("CREATE VIEW %s AS SELECT 1 AS id", "test");
+ assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false));
+ }
+
private Table getTable(String name) {
return validationCatalog.loadTable(TableIdentifier.of("default", name));
}
diff --git
a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
index 011626c280..c5bd648f2e 100644
---
a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
+++
b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
@@ -66,10 +66,13 @@ case class RewriteViewCommands(spark: SparkSession) extends
Rule[LogicalPlan] wi
allowExisting = allowExisting,
replace = replace)
- case ShowViews(UnresolvedNamespace(Seq()), pattern, output)
- if ViewUtil.isViewCatalog(catalogManager.currentCatalog) =>
- ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog,
catalogManager.currentNamespace),
- pattern, output)
+ case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) =>
+ if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) {
+ ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog,
catalogManager.currentNamespace),
+ pattern, output)
+ } else {
+ view
+ }
case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)),
pattern, output)
if ViewUtil.isViewCatalog(catalog) =>
diff --git
a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
index 3cc1e32d00..d521cc72cb 100644
---
a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
+++
b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java
@@ -1462,6 +1462,11 @@ public class TestViews extends ExtensionsTestBase {
.contains(
// spark stores temp views case-insensitive by default
row("global_temp", "globalviewforlisting", true), tempView);
+
+ sql("USE spark_catalog");
+ assertThat(sql("SHOW VIEWS")).contains(tempView);
+
+ assertThat(sql("SHOW VIEWS IN default")).contains(tempView);
}
@TestTemplate
diff --git
a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
index 207fca3cc2..4fb1a8ad06 100644
---
a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
+++
b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java
@@ -18,6 +18,8 @@
*/
package org.apache.iceberg.spark;
+import static org.assertj.core.api.Assertions.assertThat;
+
import java.io.IOException;
import java.nio.file.Files;
import org.apache.iceberg.ParameterizedTestExtension;
@@ -174,6 +176,13 @@ public class SmokeTest extends ExtensionsTestBase {
.hasSize(3);
}
+ @TestTemplate
+ public void showView() {
+ sql("DROP VIEW IF EXISTS %s", "test");
+ sql("CREATE VIEW %s AS SELECT 1 AS id", "test");
+ assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false));
+ }
+
private Table getTable(String name) {
return validationCatalog.loadTable(TableIdentifier.of("default", name));
}