From 7b2d54c9d19f3b498ff8ecc05219db74cb0c3096 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 28 Apr 2026 23:33:42 +0000 Subject: [PATCH 01/17] [SPARK-56655][SQL] Rename v2 metadata-table API: MetadataOnlyTable, RelationCatalog, loadRelation Mechanical rename follow-up to SPARK-52729: - MetadataOnlyTable -> MetadataTable - RelationCatalog -> TableViewCatalog - loadRelation() -> loadTableOrView() Test suites renamed to track the class name: - DataSourceV2MetadataOnlyTableSuite -> DataSourceV2MetadataTableSuite - DataSourceV2MetadataOnlyViewSuite -> DataSourceV2MetadataViewSuite Other v2 helpers that share the loadRelation name (CatalogV2Util.loadRelation, RelationResolution's private loadRelation(V2TableReference)) are unrelated and left as-is. Co-authored-by: Isaac --- ...adataOnlyTable.java => MetadataTable.java} | 14 ++-- .../sql/connector/catalog/TableCatalog.java | 2 +- ...tionCatalog.java => TableViewCatalog.java} | 44 ++++++------ .../sql/connector/catalog/ViewCatalog.java | 4 +- .../spark/sql/connector/catalog/ViewInfo.java | 6 +- .../sql/catalyst/analysis/Analyzer.scala | 14 ++-- .../analysis/RelationResolution.scala | 22 +++--- .../sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../sql/catalyst/catalog/interface.scala | 2 +- .../sql/connector/catalog/Catalogs.scala | 12 ++-- .../spark/sql/connector/catalog/V1Table.scala | 2 +- .../analysis/ResolveSessionCatalog.scala | 2 +- ...a => DataSourceV2MetadataTableSuite.scala} | 20 +++--- ...la => DataSourceV2MetadataViewSuite.scala} | 70 +++++++++---------- 14 files changed, 108 insertions(+), 108 deletions(-) rename sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/{MetadataOnlyTable.java => MetadataTable.java} (84%) rename sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/{RelationCatalog.java => TableViewCatalog.java} (85%) rename sql/core/src/test/scala/org/apache/spark/sql/connector/{DataSourceV2MetadataOnlyTableSuite.scala => DataSourceV2MetadataTableSuite.scala} (90%) rename sql/core/src/test/scala/org/apache/spark/sql/connector/{DataSourceV2MetadataOnlyViewSuite.scala => DataSourceV2MetadataViewSuite.scala} (96%) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataOnlyTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataTable.java similarity index 84% rename from sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataOnlyTable.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataTable.java index b20a9b566646..0a8c29b29792 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataOnlyTable.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataTable.java @@ -33,17 +33,17 @@ * (for views) at read time. *

* Catalogs build the metadata via {@link TableInfo.Builder} (for data-source tables) or - * {@link ViewInfo.Builder} (for views). A {@code MetadataOnlyTable} wrapping a + * {@link ViewInfo.Builder} (for views). A {@code MetadataTable} wrapping a * {@link TableInfo} can be returned from {@link TableCatalog#loadTable(Identifier)} for a - * data-source table; a {@code MetadataOnlyTable} wrapping a {@link ViewInfo} can be returned - * from {@link RelationCatalog#loadRelation(Identifier)} as the single-RPC perf opt-in for a view. + * data-source table; a {@code MetadataTable} wrapping a {@link ViewInfo} can be returned + * from {@link TableViewCatalog#loadTableOrView(Identifier)} as the single-RPC perf opt-in for a view. * Downstream consumers distinguish the two by checking * {@code getTableInfo() instanceof ViewInfo}. * * @since 4.2.0 */ @Evolving -public class MetadataOnlyTable implements Table { +public class MetadataTable implements Table { private final TableInfo info; private final String name; @@ -51,12 +51,12 @@ public class MetadataOnlyTable implements Table { * @param info metadata for the table or view. Pass a {@link ViewInfo} for a view. * @param name human-readable name for this table, used by places that read {@link #name()} * (e.g. the {@code Name} row of {@code DESCRIBE TABLE EXTENDED}). Catalogs - * returning a {@code MetadataOnlyTable} from {@link TableCatalog#loadTable} or - * {@link RelationCatalog#loadRelation} should typically pass + * returning a {@code MetadataTable} from {@link TableCatalog#loadTable} or + * {@link TableViewCatalog#loadTableOrView} should typically pass * {@code ident.toString()}, matching the quoted multi-part form used elsewhere * for v2 identifiers. */ - public MetadataOnlyTable(TableInfo info, String name) { + public MetadataTable(TableInfo info, String name) { this.info = Objects.requireNonNull(info, "info should not be null"); this.name = Objects.requireNonNull(name, "name should not be null"); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index 55894357f19d..a6f51342aef5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -34,7 +34,7 @@ * Catalog API for connectors that expose tables. *

* Connectors that expose only tables implement this interface. Connectors that expose - * both tables and views must implement {@link RelationCatalog} (which extends both this + * both tables and views must implement {@link TableViewCatalog} (which extends both this * interface and {@link ViewCatalog} and adds the cross-cutting contract for the combined * case); the methods on this interface remain table-only -- they do not interact with views. *

diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/RelationCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableViewCatalog.java similarity index 85% rename from sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/RelationCatalog.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableViewCatalog.java index bb674faa10ac..a24ec8e357e2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/RelationCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableViewCatalog.java @@ -27,15 +27,15 @@ * Catalog API for connectors that expose both tables and views in a single shared identifier * namespace. *

- * Connectors that expose both tables and views must implement {@code RelationCatalog}; + * Connectors that expose both tables and views must implement {@code TableViewCatalog}; * implementing {@link TableCatalog} and {@link ViewCatalog} directly without - * {@code RelationCatalog} is rejected at catalog initialization. Connectors that expose only + * {@code TableViewCatalog} is rejected at catalog initialization. Connectors that expose only * tables implement just {@link TableCatalog}; connectors that expose only views implement just * {@link ViewCatalog}; this interface is not relevant to them. * *

Two principles

* - * A {@code RelationCatalog} follows two rules that, taken together, define every cross-cutting + * A {@code TableViewCatalog} follows two rules that, taken together, define every cross-cutting * subtlety: *
    *
  1. Orthogonal interfaces. Every {@link TableCatalog} method behaves as if views did @@ -97,11 +97,11 @@ *

    Single-RPC perf entry points

    * * The orthogonal {@link TableCatalog} and {@link ViewCatalog} answer two cross-cutting - * questions in two round trips each. {@code RelationCatalog} adds dedicated methods so a + * questions in two round trips each. {@code TableViewCatalog} adds dedicated methods so a * catalog can answer both in one round trip: *
      - *
    • {@link #loadRelation(Identifier)} -- the resolver's per-identifier read path. Returns - * a regular {@link Table} for a table, or a {@link MetadataOnlyTable} wrapping a + *
    • {@link #loadTableOrView(Identifier)} -- the resolver's per-identifier read path. Returns + * a regular {@link Table} for a table, or a {@link MetadataTable} wrapping a * {@link ViewInfo} for a view. Saves the {@code loadTable} -> {@code loadView} fallback * on a cold cache.
    • *
    • {@link #listRelationSummaries(String[])} -- a unified listing of tables and views with the @@ -113,22 +113,22 @@ * @since 4.2.0 */ @Evolving -public interface RelationCatalog extends TableCatalog, ViewCatalog { +public interface TableViewCatalog extends TableCatalog, ViewCatalog { /** * Load metadata for an identifier that may resolve to either a table or a view. *

      * For a table, returns the table's {@link Table}. For a view, returns a - * {@link MetadataOnlyTable} wrapping a {@link ViewInfo}; callers discriminate via + * {@link MetadataTable} wrapping a {@link ViewInfo}; callers discriminate via * {@code getTableInfo() instanceof ViewInfo}. This lets the resolver answer in a single RPC * instead of falling back from {@link TableCatalog#loadTable} to {@link ViewCatalog#loadView}. * * @param ident the identifier - * @return a {@link Table} for tables, or a {@link MetadataOnlyTable} wrapping a + * @return a {@link Table} for tables, or a {@link MetadataTable} wrapping a * {@link ViewInfo} for views * @throws NoSuchTableException if neither a table nor a view exists at {@code ident} */ - Table loadRelation(Identifier ident) throws NoSuchTableException; + Table loadTableOrView(Identifier ident) throws NoSuchTableException; /** * List the tables and views in a namespace, returned as {@link TableSummary} entries with @@ -162,14 +162,14 @@ default TableSummary[] listRelationSummaries(String[] namespace) /** * {@inheritDoc} *

      - * The default implementation derives from {@link #loadRelation}: a {@link MetadataOnlyTable} + * The default implementation derives from {@link #loadTableOrView}: a {@link MetadataTable} * wrapping a {@link ViewInfo} is rejected as not-a-table; anything else is returned. Override * only if a tables-only path is materially cheaper than the unified one. */ @Override default Table loadTable(Identifier ident) throws NoSuchTableException { - Table t = loadRelation(ident); - if (t instanceof MetadataOnlyTable mot && mot.getTableInfo() instanceof ViewInfo) { + Table t = loadTableOrView(ident); + if (t instanceof MetadataTable mot && mot.getTableInfo() instanceof ViewInfo) { throw new NoSuchTableException(ident); } return t; @@ -178,7 +178,7 @@ default Table loadTable(Identifier ident) throws NoSuchTableException { /** * {@inheritDoc} *

      - * The default implementation derives from {@link #loadRelation}: a {@link MetadataOnlyTable} + * The default implementation derives from {@link #loadTableOrView}: a {@link MetadataTable} * wrapping a {@link ViewInfo} is unwrapped and returned; anything else (table or absent) is * surfaced as {@link NoSuchViewException}. Override only if a views-only path is materially * cheaper than the unified one. @@ -187,11 +187,11 @@ default Table loadTable(Identifier ident) throws NoSuchTableException { default ViewInfo loadView(Identifier ident) throws NoSuchViewException { Table t; try { - t = loadRelation(ident); + t = loadTableOrView(ident); } catch (NoSuchTableException e) { throw new NoSuchViewException(ident); } - if (t instanceof MetadataOnlyTable mot && mot.getTableInfo() instanceof ViewInfo vi) { + if (t instanceof MetadataTable mot && mot.getTableInfo() instanceof ViewInfo vi) { return vi; } throw new NoSuchViewException(ident); @@ -200,14 +200,14 @@ default ViewInfo loadView(Identifier ident) throws NoSuchViewException { /** * {@inheritDoc} *

      - * The default implementation derives from {@link #loadRelation}: returns {@code true} only if + * The default implementation derives from {@link #loadTableOrView}: returns {@code true} only if * the entry exists and is not a view. Override only if a cheaper existence-check path exists. */ @Override default boolean tableExists(Identifier ident) { try { - Table t = loadRelation(ident); - return !(t instanceof MetadataOnlyTable mot && mot.getTableInfo() instanceof ViewInfo); + Table t = loadTableOrView(ident); + return !(t instanceof MetadataTable mot && mot.getTableInfo() instanceof ViewInfo); } catch (NoSuchTableException e) { return false; } @@ -216,14 +216,14 @@ default boolean tableExists(Identifier ident) { /** * {@inheritDoc} *

      - * The default implementation derives from {@link #loadRelation}: returns {@code true} only if + * The default implementation derives from {@link #loadTableOrView}: returns {@code true} only if * the entry exists and is a view. Override only if a cheaper existence-check path exists. */ @Override default boolean viewExists(Identifier ident) { try { - Table t = loadRelation(ident); - return t instanceof MetadataOnlyTable mot && mot.getTableInfo() instanceof ViewInfo; + Table t = loadTableOrView(ident); + return t instanceof MetadataTable mot && mot.getTableInfo() instanceof ViewInfo; } catch (NoSuchTableException e) { return false; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java index 184676023d7c..be2bc8dacc4d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java @@ -25,7 +25,7 @@ * Catalog API for connectors that expose views. *

      * Connectors that expose only views implement this interface. Connectors that expose - * both tables and views must implement {@link RelationCatalog} (which extends both this + * both tables and views must implement {@link TableViewCatalog} (which extends both this * interface and {@link TableCatalog} and adds the cross-cutting contract for the combined * case); the methods on this interface remain view-only -- they do not interact with tables. *

      @@ -126,7 +126,7 @@ ViewInfo createView(Identifier ident, ViewInfo info) * concurrent {@code CREATE VIEW} won the race in the * default impl's gap between {@link #replaceView} and * the fallback {@link #createView}, or, in a - * {@link RelationCatalog}, a table sits at {@code ident} + * {@link TableViewCatalog}, a table sits at {@code ident} * @throws NoSuchNamespaceException if the identifier's namespace does not exist (optional) */ default ViewInfo createOrReplaceView(Identifier ident, ViewInfo info) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java index da82de01f8e4..f70e3649b3f6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java @@ -31,9 +31,9 @@ * query output column names. Schema and user TBLPROPERTIES are inherited from {@link TableInfo} * via the typed builder. *

      - * {@code ViewInfo} extends {@link TableInfo} so that a {@link RelationCatalog} can opt into the - * single-RPC perf path by returning a {@link MetadataOnlyTable} wrapping a {@code ViewInfo} - * from {@link RelationCatalog#loadRelation} for a view identifier. Pure {@link ViewCatalog} + * {@code ViewInfo} extends {@link TableInfo} so that a {@link TableViewCatalog} can opt into the + * single-RPC perf path by returning a {@link MetadataTable} wrapping a {@code ViewInfo} + * from {@link TableViewCatalog#loadTableOrView} for a view identifier. Pure {@link ViewCatalog} * implementations never see {@code TableInfo}; the typed setters on {@link Builder} cover * everything they need to construct a {@code ViewInfo}. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 323a7db9c7ad..af537ec9f2f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1132,8 +1132,8 @@ class Analyzer( * so surfacing a downstream "view not found" would hide the real reason. * * Lookup order against a non-session catalog: - * 1. If the catalog is a [[RelationCatalog]], [[RelationCatalog.loadRelation]] is called - * once. A returned [[MetadataOnlyTable]] wrapping a [[ViewInfo]] is interpreted as a + * 1. If the catalog is a [[TableViewCatalog]], [[TableViewCatalog.loadTableOrView]] is called + * once. A returned [[MetadataTable]] wrapping a [[ViewInfo]] is interpreted as a * view; other results are tables. * 2. Otherwise, [[TableCatalog.loadTable]] is tried (when implemented), then * [[ViewCatalog.loadView]] as the fallback view-resolution path (when implemented). @@ -1151,13 +1151,13 @@ class Analyzer( throw QueryCompilationErrors.missingCatalogViewsAbilityError(catalog) } catalog match { - case mc: RelationCatalog => - // Single-RPC perf path: loadRelation returns a Table for a table or a - // MetadataOnlyTable wrapping a ViewInfo for a view. NoSuchTable means + case mc: TableViewCatalog => + // Single-RPC perf path: loadTableOrView returns a Table for a table or a + // MetadataTable wrapping a ViewInfo for a view. NoSuchTable means // neither exists. try { - Some(mc.loadRelation(ident) match { - case t: MetadataOnlyTable if t.getTableInfo.isInstanceOf[ViewInfo] => + Some(mc.loadTableOrView(ident) match { + case t: MetadataTable if t.getTableInfo.isInstanceOf[ViewInfo] => ResolvedPersistentView( catalog, ident, V1Table.toCatalogTable(catalog, ident, t)) case table => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala index 7a5077a8a3e1..184c77d58547 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala @@ -37,10 +37,10 @@ import org.apache.spark.sql.connector.catalog.{ ChangelogInfo, Identifier, LookupCatalog, - MetadataOnlyTable, - RelationCatalog, + MetadataTable, Table, TableCatalog, + TableViewCatalog, V1Table, V2TableWithV1Fallback, ViewCatalog, @@ -262,8 +262,8 @@ class RelationResolution( .orElse { val writePrivileges = u.options.get(UnresolvedRelation.REQUIRED_WRITE_PRIVILEGES) val finalOptions = u.clearWritePrivileges.options - // For a `RelationCatalog` with no time-travel / write privileges, the single-RPC - // `loadRelation` answers both "is there a table?" and "is there a view?" in one + // For a `TableViewCatalog` with no time-travel / write privileges, the single-RPC + // `loadTableOrView` answers both "is there a table?" and "is there a view?" in one // call. Time-travel and write privileges apply to tables only, so for those the // lookup falls through to the table-only `loadTable` path below; views are not // reachable via the v2 fallback in those cases. @@ -272,9 +272,9 @@ class RelationResolution( // mixin): `CatalogV2Util.loadTable` would call `asTableCatalog` and throw // MISSING_CATALOG_ABILITY.TABLES, masking the legitimate view-resolution path. val tableOrView: Option[Table] = catalog match { - case mc: RelationCatalog if finalTimeTravelSpec.isEmpty && writePrivileges == null => + case mc: TableViewCatalog if finalTimeTravelSpec.isEmpty && writePrivileges == null => try { - Some(mc.loadRelation(ident)) + Some(mc.loadTableOrView(ident)) } catch { case _: NoSuchTableException => None } @@ -299,7 +299,7 @@ class RelationResolution( catalog match { case vc: ViewCatalog => try { - Some(new MetadataOnlyTable(vc.loadView(ident), ident.toString)) + Some(new MetadataTable(vc.loadView(ident), ident.toString)) } catch { case _: NoSuchViewException => None } @@ -313,7 +313,7 @@ class RelationResolution( // `table` is `tableOrView` filtered to tables only -- used for cache lookup since // we don't share-cache views. val table: Option[Table] = tableOrView.filter { - case t: MetadataOnlyTable if t.getTableInfo.isInstanceOf[ViewInfo] => false + case t: MetadataTable if t.getTableInfo.isInstanceOf[ViewInfo] => false case _ => true } @@ -426,10 +426,10 @@ class RelationResolution( || !v1Table.catalogTable.tracksPartitionsInCatalog => createDataSourceV1Scan(v1Table.v1Table) - // MetadataOnlyTable is a sentinel meaning "interpret via v1", so unlike the V1Table + // MetadataTable is a sentinel meaning "interpret via v1", so unlike the V1Table // case above we apply no session-catalog / tracksPartitionsInCatalog guard -- any catalog - // returning MetadataOnlyTable has opted into v1 read semantics. - case t: MetadataOnlyTable => + // returning MetadataTable has opted into v1 read semantics. + case t: MetadataTable => createDataSourceV1Scan(V1Table.toCatalogTable(catalog, ident, t)) case table => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index af398eb8527e..a3efade9b9a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1058,7 +1058,7 @@ class SessionCatalog( // so the SubqueryAlias qualifier reflects the real catalog + multi-part namespace. // Fall back to the historical 3-part form for v1 session-catalog tables -- we intentionally // always include `SESSION_CATALOG_NAME` here and ignore - // `LEGACY_NON_IDENTIFIER_OUTPUT_CATALOG_NAME` to preserve pre-v2-MetadataOnlyTable behavior. + // `LEGACY_NON_IDENTIFIER_OUTPUT_CATALOG_NAME` to preserve pre-v2-MetadataTable behavior. val multiParts = metadata.multipartIdentifier.getOrElse { val qualifiedIdent = qualifyIdentifier(metadata.identifier) Seq(CatalogManager.SESSION_CATALOG_NAME, qualifiedIdent.database.get, qualifiedIdent.table) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 981b2ac96a37..1c4362bfd3ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -447,7 +447,7 @@ case class CatalogTable( ignoredProperties: Map[String, String] = Map.empty, viewOriginalText: Option[String] = None, // Multi-part identifier [catalog, namespace..., name] for tables synthesized from a v2 - // `MetadataOnlyTable` whose namespace has more than one part -- the v1 `identifier: + // `MetadataTable` whose namespace has more than one part -- the v1 `identifier: // TableIdentifier` (single-string database) cannot carry that losslessly. `None` for // v1-native tables; callers should use `fullIdent` which falls back to `identifier.nameParts`. multipartIdentifier: Option[Seq[String]] = None) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala index 03addeb17069..c40d5ab67919 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala @@ -64,7 +64,7 @@ private[sql] object Catalogs { } val plugin = pluginClass.getDeclaredConstructor().newInstance().asInstanceOf[CatalogPlugin] plugin.initialize(name, catalogOptions(name, conf)) - validateRelationCatalog(name, plugin) + validateTableViewCatalog(name, plugin) plugin } catch { case e: ClassNotFoundException => @@ -110,17 +110,17 @@ private[sql] object Catalogs { /** * Reject catalogs that implement both [[TableCatalog]] and [[ViewCatalog]] without - * extending [[RelationCatalog]]. The combined case has cross-cutting rules (single namespace, - * cross-type collision rejection, perf opt-ins) that live on [[RelationCatalog]]; implementing + * extending [[TableViewCatalog]]. The combined case has cross-cutting rules (single namespace, + * cross-type collision rejection, perf opt-ins) that live on [[TableViewCatalog]]; implementing * the two interfaces directly would skip that contract. */ - private def validateRelationCatalog(name: String, plugin: CatalogPlugin): Unit = { + private def validateTableViewCatalog(name: String, plugin: CatalogPlugin): Unit = { if (plugin.isInstanceOf[TableCatalog] && plugin.isInstanceOf[ViewCatalog] && - !plugin.isInstanceOf[RelationCatalog]) { + !plugin.isInstanceOf[TableViewCatalog]) { throw new IllegalArgumentException( s"Catalog '$name' (${plugin.getClass.getName}) implements both TableCatalog and " + s"ViewCatalog directly. Catalogs that expose both tables and views must implement " + - s"RelationCatalog instead, which centralizes the cross-cutting rules (shared " + + s"TableViewCatalog instead, which centralizes the cross-cutting rules (shared " + s"identifier namespace, cross-type collision rejection, single-RPC perf entry " + s"points).") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala index 079b2639aa2b..a1fb2c1c84e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala @@ -112,7 +112,7 @@ private[sql] object V1Table { def toCatalogTable( catalog: CatalogPlugin, ident: Identifier, - t: MetadataOnlyTable): CatalogTable = t.getTableInfo match { + t: MetadataTable): CatalogTable = t.getTableInfo match { case viewInfo: ViewInfo => toCatalogTable(catalog, ident, viewInfo) case tableInfo => toCatalogTable(catalog, ident, tableInfo) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 94523dd313b4..f8a9ce1d40d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -786,7 +786,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) object ResolvedViewIdentifier { // Only matches session-catalog persistent views. Non-session-catalog persistent views - // (produced for `MetadataOnlyTable`) fall through; `AlterViewAs` is picked up by the v2 + // (produced for `MetadataTable`) fall through; `AlterViewAs` is picked up by the v2 // strategy, and the remaining view DDL / inspection plans (SET/UNSET TBLPROPERTIES, // ALTER VIEW ... WITH SCHEMA, RENAME TO, SHOW CREATE TABLE, SHOW TBLPROPERTIES, SHOW // COLUMNS, DESCRIBE [COLUMN]) are rejected with `UNSUPPORTED_FEATURE.TABLE_OPERATION` by diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataOnlyTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataTableSuite.scala similarity index 90% rename from sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataOnlyTableSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataTableSuite.scala index 8d3ad19419df..c520ea451ec4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataOnlyTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataTableSuite.scala @@ -20,18 +20,18 @@ package org.apache.spark.sql.connector import org.apache.spark.SparkConf import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException -import org.apache.spark.sql.connector.catalog.{Identifier, MetadataOnlyTable, Table, TableCatalog, TableChange, TableInfo, TableSummary} +import org.apache.spark.sql.connector.catalog.{Identifier, MetadataTable, Table, TableCatalog, TableChange, TableInfo, TableSummary} import org.apache.spark.sql.connector.expressions.LogicalExpressions import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap /** - * Tests for the data-source-table side of [[MetadataOnlyTable]]: a v2 catalog returns + * Tests for the data-source-table side of [[MetadataTable]]: a v2 catalog returns * metadata-only tables and Spark reads / writes them via the V1 data-source path. - * View-related paths live in [[DataSourceV2MetadataOnlyViewSuite]]. + * View-related paths live in [[DataSourceV2MetadataViewSuite]]. */ -class DataSourceV2MetadataOnlyTableSuite extends QueryTest with SharedSparkSession { +class DataSourceV2MetadataTableSuite extends QueryTest with SharedSparkSession { import testImplicits._ override def sparkConf: SparkConf = super.sparkConf @@ -83,8 +83,8 @@ class DataSourceV2MetadataOnlyTableSuite extends QueryTest with SharedSparkSessi checkAnswer(spark.table(tableName), 0.until(10).map(i => Row(i, -i))) } - test("DESCRIBE TABLE EXTENDED on a non-view MetadataOnlyTable shows the real identifier") { - // MetadataOnlyTable.name() is read by DescribeTableExec's "Name" row. Pin that it + test("DESCRIBE TABLE EXTENDED on a non-view MetadataTable shows the real identifier") { + // MetadataTable.name() is read by DescribeTableExec's "Name" row. Pin that it // reflects the catalog-supplied identifier (here TestingDataSourceTableCatalog passes // `ident.toString`) rather than a generic placeholder, so the DESCRIBE output is // meaningful for users. @@ -129,7 +129,7 @@ class DataSourceV2MetadataOnlyTableSuite extends QueryTest with SharedSparkSessi } /** - * A read-only [[TableCatalog]] that returns [[MetadataOnlyTable]] for a small set of canned + * A read-only [[TableCatalog]] that returns [[MetadataTable]] for a small set of canned * table fixtures. Used to drive the data-source-table read path (file source + v2 provider) * through Spark's V1 data-source machinery. */ @@ -142,7 +142,7 @@ class TestingDataSourceTableCatalog extends TableCatalog { .withLocation(ident.namespace().head) .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) .build() - new MetadataOnlyTable(info, ident.toString) + new MetadataTable(info, ident.toString) case "test_partitioned_json" => val partitioning = LogicalExpressions.identity(LogicalExpressions.reference(Seq("c2"))) val info = new TableInfo.Builder() @@ -152,13 +152,13 @@ class TestingDataSourceTableCatalog extends TableCatalog { .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) .withPartitions(Array(partitioning)) .build() - new MetadataOnlyTable(info, ident.toString) + new MetadataTable(info, ident.toString) case "test_v2" => val info = new TableInfo.Builder() .withSchema(FakeV2Provider.schema) .withProvider(classOf[FakeV2Provider].getName) .build() - new MetadataOnlyTable(info, ident.toString) + new MetadataTable(info, ident.toString) case _ => throw new NoSuchTableException(ident) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataOnlyViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataOnlyViewSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala index 0851e6d2df76..a24f71884ed7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataOnlyViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala @@ -20,31 +20,31 @@ package org.apache.spark.sql.connector import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, NoSuchViewException, TableAlreadyExistsException, ViewAlreadyExistsException} -import org.apache.spark.sql.connector.catalog.{Identifier, MetadataOnlyTable, RelationCatalog, Table, TableCatalog, TableChange, TableInfo, TableSummary, V1Table, ViewCatalog, ViewInfo} +import org.apache.spark.sql.connector.catalog.{Identifier, MetadataTable, Table, TableCatalog, TableChange, TableInfo, TableSummary, TableViewCatalog, V1Table, ViewCatalog, ViewInfo} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap /** - * Tests for the view side of [[MetadataOnlyTable]]: view-text expansion on read, and + * Tests for the view side of [[MetadataTable]]: view-text expansion on read, and * CREATE VIEW / ALTER VIEW ... AS going through the v2 write path * (`CreateV2ViewExec` / `AlterV2ViewExec`). View writes route through * [[ViewCatalog#createView]] / [[ViewCatalog#replaceView]]. * Data-source-table read paths live in - * [[org.apache.spark.sql.connector.DataSourceV2MetadataOnlyTableSuite]]. + * [[org.apache.spark.sql.connector.DataSourceV2MetadataTableSuite]]. * * TODO: once the remaining v2 view DDL is implemented (SET/UNSET TBLPROPERTIES, SHOW CREATE * VIEW, RENAME TO, SCHEMA BINDING, DESCRIBE / SHOW TBLPROPERTIES on v2 views), register a - * `MetadataOnlyTable`-backed `DelegatingCatalogExtension` as `spark.sql.catalog.spark_catalog` + * `MetadataTable`-backed `DelegatingCatalogExtension` as `spark.sql.catalog.spark_catalog` * and run the shared [[org.apache.spark.sql.execution.PersistedViewTestSuite]] body against * the v2 path for full parity with the v1 persisted-view coverage. */ -class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSession { +class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { import testImplicits._ override def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.catalog.view_catalog", classOf[TestingRelationCatalog].getName) + .set("spark.sql.catalog.view_catalog", classOf[TestingTableViewCatalog].getName) // --- View read path ----------------------------------------------------- @@ -72,7 +72,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio // End-to-end coverage of the v2 encoder -> parser round-trip: test_unqualified_multi is a // view whose captured catalog+namespace is view_catalog.ns1.ns2 (two-part namespace) and // whose body references `t` unqualified. At read time the unqualified `t` must expand to - // view_catalog.ns1.ns2.t via the captured context -- which TestingRelationCatalog resolves to + // view_catalog.ns1.ns2.t via the captured context -- which TestingTableViewCatalog resolves to // its own `t` fixture at that namespace. checkAnswer( spark.table("view_catalog.outer_ns.test_unqualified_multi"), @@ -93,7 +93,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio .withCurrentCatalog("my_cat") .withCurrentNamespace(Array("db1", "db2")) .build() - val motTable = new MetadataOnlyTable(info, "v") + val motTable = new MetadataTable(info, "v") // Any CatalogPlugin works here; toCatalogTable only reads `catalog.name()`. val catalog = spark.sessionState.catalogManager.catalog("view_catalog") val ct = V1Table.toCatalogTable( @@ -108,7 +108,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio .withCurrentNamespace(Array("weird.db", "normal")) .build() val ctWeird = V1Table.toCatalogTable( - catalog, Identifier.of(Array("ns"), "v"), new MetadataOnlyTable(infoWeird, "v")) + catalog, Identifier.of(Array("ns"), "v"), new MetadataTable(infoWeird, "v")) assert(ctWeird.viewCatalogAndNamespace == Seq("my_cat", "weird.db", "normal")) } @@ -117,7 +117,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio .withSchema(new StructType().add("col", "string")) .withQueryText("SELECT * FROM spark_catalog.default.t") .build() - val motTable = new MetadataOnlyTable(info, "v") + val motTable = new MetadataTable(info, "v") val catalog = spark.sessionState.catalogManager.catalog("view_catalog") val ct = V1Table.toCatalogTable(catalog, Identifier.of(Array("ns"), "v"), motTable) assert(ct.viewCatalogAndNamespace.isEmpty) @@ -232,10 +232,10 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio Seq("a", "b").toDF("col").write.saveAsTable("spark_catalog.default.t") sql("CREATE VIEW view_catalog.default.v_coll DEFAULT COLLATION UTF8_BINARY AS " + "SELECT col FROM spark_catalog.default.t") - // TestingRelationCatalog stores the TableInfo verbatim, so the collation property is + // TestingTableViewCatalog stores the TableInfo verbatim, so the collation property is // observable via the catalog-stored builder output. val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] val info = catalog.getStoredView(Array("default"), "v_coll") assert(info.properties().get(TableCatalog.PROP_COLLATION) == "UTF8_BINARY") } @@ -258,7 +258,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio test("CREATE VIEW over a non-view table entry is rejected (plain TableCatalog)") { val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] val tableIdent = Identifier.of(Array("default"), "v_existing_table") val tableInfo = new TableInfo.Builder() .withSchema(new StructType().add("col", "string")) @@ -374,7 +374,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio "SELECT x + 1 AS x FROM spark_catalog.default.t") val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] val info = catalog.getStoredView(Array("default"), "v_preserve") assert(info.properties().get("mykey") == "myvalue") } @@ -387,7 +387,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio "SELECT x FROM spark_catalog.default.t") val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] val info = catalog.getStoredView(Array("default"), "v_owner_create") // v2 CREATE VIEW stamps the current user into PROP_OWNER, matching v2 CREATE TABLE // (via CatalogV2Util.withDefaultOwnership) and v1 CREATE VIEW (via CatalogTable.owner's @@ -400,7 +400,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio test("ALTER VIEW preserves PROP_OWNER (v1-parity)") { val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] val viewIdent = Identifier.of(Array("default"), "v_owner") // Pre-seed a view whose stored ViewInfo carries an explicit owner. val initialInfo = new ViewInfo.Builder() @@ -435,7 +435,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio "SELECT x + 1 AS x FROM spark_catalog.default.t") val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] assert(catalog.getStoredView(Array("default"), "v_evo").schemaMode() == "EVOLUTION") } } @@ -448,7 +448,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio "SELECT col FROM spark_catalog.default.t") } val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] assert(catalog.getStoredView(Array("default"), "v_configs") .sqlConfigs().get(SQLConf.ANSI_ENABLED.key) == "true") @@ -480,7 +480,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio // ALTER VIEW's identifier is resolved via `UnresolvedView`, whose `viewOnly=true` path // in `Analyzer.lookupTableOrView` rejects non-ViewCatalog catalogs up front with the // expected error class -- before `loadTable` is even called. `TestingTableOnlyCatalog` - // happens to round-trip `default.v` as a view-typed MetadataOnlyTable, but that fixture + // happens to round-trip `default.v` as a view-typed MetadataTable, but that fixture // is not actually consulted on this path. CREATE VIEW's capability check lives in // `CheckViewReferences`; ALTER VIEW's lives in the analyzer gate. Both yield // `MISSING_CATALOG_ABILITY.VIEWS`. @@ -571,7 +571,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio // `unsupportedCreateOrReplaceViewOnTableError`. Pre-seed a non-view entry at a // multi-level-namespace identifier to exercise the rendering. val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] val tblIdent = Identifier.of(Array("ns1", "inner"), "t_err") catalog.createTable( tblIdent, @@ -742,7 +742,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio test("DROP VIEW on a ViewCatalog drops the view") { val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] withTable("spark_catalog.default.t") { Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") sql("CREATE VIEW view_catalog.default.v_drop AS " + @@ -764,7 +764,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio // `wrongCommandForObjectTypeError`. The v2 path must also refuse -- otherwise // `DROP VIEW view_catalog.default.` would silently destroy the table's entry. val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] val tableIdent = Identifier.of(Array("default"), "t_not_a_view") catalog.createTable( tableIdent, @@ -799,7 +799,7 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio private def seedV2Table(name: String): Unit = { val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingRelationCatalog] + .asInstanceOf[TestingTableViewCatalog] catalog.createTable( Identifier.of(Array("default"), name), new TableInfo.Builder() @@ -869,26 +869,26 @@ class DataSourceV2MetadataOnlyViewSuite extends QueryTest with SharedSparkSessio } /** - * A [[RelationCatalog]]: round-trips [[MetadataOnlyTable]] for created views and tables and + * A [[TableViewCatalog]]: round-trips [[MetadataTable]] for created views and tables and * exposes a few canned read-only view fixtures (`test_view`, `test_unqualified_view`, * `test_unqualified_multi`, plus an unqualified-target view at `ns1.ns2.t`) used by the * view-read tests. Entries created via `createTable` / `createView` are distinguished by the * stored value's runtime type (ViewInfo vs TableInfo). The single-RPC perf entry point - * [[loadRelation]] returns either kind; [[loadTable]] is tables-only per the + * [[loadTableOrView]] returns either kind; [[loadTable]] is tables-only per the * [[TableCatalog#loadTable]] contract. */ -class TestingRelationCatalog extends RelationCatalog { +class TestingTableViewCatalog extends TableViewCatalog { // Holds entries (views and tables) created via createTable / createView within the session. // Keyed by (namespace, name); the stored value's runtime type (ViewInfo vs TableInfo) // distinguishes views from tables. Mixed-catalog: shared identifier namespace per the - // RelationCatalog contract. + // TableViewCatalog contract. private val createdViews = new java.util.concurrent.ConcurrentHashMap[(Seq[String], String), TableInfo]() - // Canned read-only view fixtures, exposed only via the perf path (loadRelation). loadView - // does not need to expose them because the resolver routes RelationCatalog reads through - // loadRelation. + // Canned read-only view fixtures, exposed only via the perf path (loadTableOrView). loadView + // does not need to expose them because the resolver routes TableViewCatalog reads through + // loadTableOrView. private def fixtureView(ident: Identifier): Option[ViewInfo] = ident.name() match { case "test_view" => Some(new ViewInfo.Builder() @@ -925,15 +925,15 @@ class TestingRelationCatalog extends RelationCatalog { case _ => None } - override def loadRelation(ident: Identifier): Table = { - // Single-RPC perf path: returns tables AND views (as MetadataOnlyTable). Stored entries + override def loadTableOrView(ident: Identifier): Table = { + // Single-RPC perf path: returns tables AND views (as MetadataTable). Stored entries // win over fixture views (the fixture namespace is read-only and disjoint from // createdViews in practice). loadTable, loadView, tableExists, viewExists all derive - // from this via the RelationCatalog default impls. + // from this via the TableViewCatalog default impls. val key = (ident.namespace().toSeq, ident.name()) Option(createdViews.get(key)) .orElse(fixtureView(ident)) - .map(new MetadataOnlyTable(_, ident.toString)) + .map(new MetadataTable(_, ident.toString)) .getOrElse(throw new NoSuchTableException(ident)) } @@ -945,7 +945,7 @@ class TestingRelationCatalog extends RelationCatalog { if (createdViews.putIfAbsent(key, info) != null) { throw new TableAlreadyExistsException(ident) } - new MetadataOnlyTable(info, ident.toString) + new MetadataTable(info, ident.toString) } /** Test-only accessor: returns the stored TableInfo (table or view) for the identifier. */ From b27919c2dcb471be589630ce4b84d4134c0551d7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 28 Apr 2026 23:57:25 +0000 Subject: [PATCH 02/17] [SPARK-56655][SQL] Implement remaining v2 view DDL and inspection commands Follow-up to SPARK-52729: close out the *Remaining work* section of the parent PR. Write-side execs (`AlterV2ViewExec.scala`): - ALTER VIEW ... SET TBLPROPERTIES (`AlterV2ViewSetPropertiesExec`) - ALTER VIEW ... UNSET TBLPROPERTIES (`AlterV2ViewUnsetPropertiesExec`) - ALTER VIEW ... WITH SCHEMA (`AlterV2ViewSchemaBindingExec`) - ALTER VIEW ... RENAME TO (`RenameV2ViewExec`) -- backed by a new abstract `ViewCatalog#renameView(Identifier, Identifier)`. The strategy consults the parser-set `isView` flag and rejects ALTER TABLE syntax on a view with EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW (mirrors v1 `DDLUtils.verifyAlterTableType`). Cross-type rejection contract for `renameView` is added to `TableViewCatalog`'s docs. - A shared `V2ViewMetadataMutation.builderFrom(existing)` helper seeds a `ViewInfo.Builder` from the existing view so mutating execs override only the field they're changing -- schema, queryText, captured resolution context, captured SQL configs, and queryColumnNames flow through unchanged. Read-side execs (`V2ViewInspectionExecs.scala`): - SHOW CREATE TABLE / SHOW TBLPROPERTIES / SHOW COLUMNS / DESCRIBE TABLE on a v2 view -- consume the typed `ViewInfo` resolved at analysis time and format output rows directly. DESCRIBE TABLE EXTENDED emits a v2-native `# Detailed View Information` block. `SHOW TABLES` on a `TableViewCatalog` now routes through `listRelationSummaries` so views appear alongside tables, matching v1 `SHOW TABLES`. Pure `TableCatalog` catalogs continue to use `listTables` and return tables only. All `UNSUPPORTED_FEATURE.TABLE_OPERATION` pins from the parent PR for these commands are replaced with real strategy cases. Architecture notes: - `ResolvedPersistentView` gains `viewInfo: Option[ViewInfo]`. Populated at resolution time by the v2 paths in `Analyzer.lookupTableOrView`. v2 execs read it directly -- no `loadView` re-fetch at exec time, matching how v2 `ResolvedTable` carries the resolved `Table`. Co-authored-by: Isaac --- .../connector/catalog/TableViewCatalog.java | 5 + .../sql/connector/catalog/ViewCatalog.java | 17 ++ .../spark/sql/connector/catalog/ViewInfo.java | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 8 +- .../analysis/ApplyDefaultCollation.scala | 18 +- .../catalyst/analysis/v2ResolutionPlans.scala | 24 +- .../sql/connector/catalog/V1ViewInfo.scala | 55 ++++ .../apache/spark/sql/classic/Catalog.scala | 8 +- .../command/DescribeRelationJsonCommand.scala | 11 +- .../spark/sql/execution/command/tables.scala | 6 +- .../datasources/v2/AlterV2ViewExec.scala | 173 +++++++++--- .../datasources/v2/DataSourceV2Strategy.scala | 97 +++---- .../datasources/v2/ShowTablesExec.scala | 19 +- .../v2/V2ViewInspectionExecs.scala | 249 ++++++++++++++++++ .../DataSourceV2MetadataViewSuite.scala | 127 +++++---- 15 files changed, 656 insertions(+), 163 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableViewCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableViewCatalog.java index a24ec8e357e2..c3298831ef3e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableViewCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableViewCatalog.java @@ -67,6 +67,9 @@ * * * + * + * + * *
      {@link org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException}
      {@link ViewCatalog#replaceView}a table sits at {@code ident}{@link org.apache.spark.sql.catalyst.analysis.NoSuchViewException}
      {@link ViewCatalog#renameView}a table sits at {@code newIdent}{@link org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException}
      * * Passive filtering (read / non-collision mutation methods that behave as if the wrong @@ -91,6 +94,8 @@ * {@link ViewCatalog#viewExists}returns {@code false} for a table * {@link ViewCatalog#dropView} * returns {@code false} for a table; does not drop it + * {@link ViewCatalog#renameView} + * throws {@code NoSuchViewException} when the source is a table * {@link ViewCatalog#listViews}views only * * diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java index be2bc8dacc4d..0e74b22079bf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java @@ -145,4 +145,21 @@ default ViewInfo createOrReplaceView(Identifier ident, ViewInfo info) * @return true if a view was dropped, false otherwise */ boolean dropView(Identifier ident); + + /** + * Rename a view. + *

      + * If the catalog supports tables and contains a table at the new identifier, this must throw + * {@link ViewAlreadyExistsException}. If the source identifier resolves to a table rather than + * a view, this must throw {@link NoSuchViewException}. The cross-type contract for catalogs + * that expose both tables and views lives on {@link TableViewCatalog}. + * + * @param oldIdent the view identifier of the existing view to rename + * @param newIdent the new view identifier + * @throws NoSuchViewException if no view exists at {@code oldIdent} + * @throws ViewAlreadyExistsException if a view (or, in a {@link TableViewCatalog}, a table) + * already exists at {@code newIdent} + */ + void renameView(Identifier oldIdent, Identifier newIdent) + throws NoSuchViewException, ViewAlreadyExistsException; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java index f70e3649b3f6..307a5ff486e5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewInfo.java @@ -49,7 +49,7 @@ public class ViewInfo extends TableInfo { private final String schemaMode; private final String[] queryColumnNames; - private ViewInfo(Builder builder) { + protected ViewInfo(Builder builder) { super(builder); this.queryText = Objects.requireNonNull(builder.queryText, "queryText should not be null"); this.currentCatalog = builder.currentCatalog; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index af537ec9f2f1..ff6feeffd0c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1159,7 +1159,7 @@ class Analyzer( Some(mc.loadTableOrView(ident) match { case t: MetadataTable if t.getTableInfo.isInstanceOf[ViewInfo] => ResolvedPersistentView( - catalog, ident, V1Table.toCatalogTable(catalog, ident, t)) + catalog, ident, t.getTableInfo.asInstanceOf[ViewInfo]) case table => ResolvedTable.create(catalog.asTableCatalog, ident, table) }) @@ -1180,7 +1180,7 @@ class Analyzer( val v1Ident = v1Table.catalogTable.identifier val v2Ident = Identifier.of(v1Ident.database.toArray, v1Ident.identifier) ResolvedPersistentView( - catalog, v2Ident, v1Table.catalogTable) + catalog, v2Ident, new V1ViewInfo(v1Table.catalogTable)) case table => ResolvedTable.create(catalog.asTableCatalog, ident, table) } @@ -1191,9 +1191,7 @@ class Analyzer( catalog match { case vc: ViewCatalog => try { - val viewInfo = vc.loadView(ident) - val catalogTable = V1Table.toCatalogTable(catalog, ident, viewInfo) - Some(ResolvedPersistentView(catalog, ident, catalogTable)) + Some(ResolvedPersistentView(catalog, ident, vc.loadView(ident))) } catch { case _: NoSuchViewException => None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala index 3e8b507e4f6c..640faf3fbe1b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.types.DataTypeUtils.{areSameBaseType, isDefaultStringCharOrVarcharType, replaceDefaultStringCharAndVarcharTypes} import org.apache.spark.sql.catalyst.util.CharVarcharUtils -import org.apache.spark.sql.connector.catalog.{SupportsNamespaces, TableCatalog} +import org.apache.spark.sql.connector.catalog.{SupportsNamespaces, TableCatalog, V1ViewInfo} import org.apache.spark.sql.types.{DataType, StringHelper, StringType} /** @@ -134,7 +134,7 @@ object ApplyDefaultCollation extends Rule[LogicalPlan] { case alterViewAs: AlterViewAs => alterViewAs.child match { case resolvedPersistentView: ResolvedPersistentView => - resolvedPersistentView.metadata.collation + Option(resolvedPersistentView.info.properties.get(TableCatalog.PROP_COLLATION)) case resolvedTempView: ResolvedTempView => resolvedTempView.metadata.collation case _ => None @@ -207,13 +207,17 @@ object ApplyDefaultCollation extends Rule[LogicalPlan] { newCreateView // We match against ResolvedPersistentView because temporary views don't have a - // schema/catalog. + // schema/catalog. Only the v1 (session-catalog) form is rewritten here -- it carries + // the underlying `CatalogTable` via `V1ViewInfo`, so we can update `collation` and + // re-wrap. v2 view paths consume `info.properties` directly and are not affected by + // this rewrite. case alterViewAs@AlterViewAs(resolvedPersistentView@ResolvedPersistentView( - catalog: SupportsNamespaces, identifier, _), _, _, _, _) - if resolvedPersistentView.metadata.collation.isEmpty => + catalog: SupportsNamespaces, identifier, v1Info: V1ViewInfo), _, _, _, _) + if v1Info.v1Table.collation.isEmpty => + val newCollation = getCollationFromSchemaMetadata(catalog, identifier.namespace()) + val newV1Table = v1Info.v1Table.copy(collation = newCollation) val newResolvedPersistentView = resolvedPersistentView.copy( - metadata = resolvedPersistentView.metadata.copy( - collation = getCollationFromSchemaMetadata(catalog, identifier.namespace()))) + info = new V1ViewInfo(newV1Table)) val newAlterViewAs = CurrentOrigin.withOrigin(alterViewAs.origin) { alterViewAs.copy(child = newResolvedPersistentView) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 046acd20a9e3..71e3c4d100dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -227,15 +227,29 @@ case class ResolvedProcedure( } /** - * A plan containing resolved persistent views. + * A plan containing a resolved persistent view. + * + * `info` is the typed v2 [[org.apache.spark.sql.connector.catalog.ViewInfo]] payload for the + * view. Session-catalog (v1) views are surfaced through the same channel via + * [[org.apache.spark.sql.connector.catalog.V1ViewInfo]], which extends `ViewInfo` and wraps + * the original [[CatalogTable]] -- mirroring the way + * [[org.apache.spark.sql.connector.catalog.V1Table]] exposes a v1 `CatalogTable` through the + * v2 [[org.apache.spark.sql.connector.catalog.Table]] surface for `ResolvedTable`. v1-only + * paths (e.g. `DescribeTableCommand`, `ShowCreateTableCommand`) recover the original + * `CatalogTable` by pattern-matching `info` against `V1ViewInfo`. */ -// TODO: create a generic representation for views, after we add view support to v2 catalog. For now -// we only hold the view schema. case class ResolvedPersistentView( catalog: CatalogPlugin, identifier: Identifier, - metadata: CatalogTable) extends LeafNodeWithoutStats { - override def output: Seq[Attribute] = Nil + info: org.apache.spark.sql.connector.catalog.ViewInfo) + extends LeafNodeWithoutStats { + // Surface the view's schema as `output` so `ResolveReferences` can resolve column references + // against it (e.g. `DescribeColumn(ResolvedPersistentView, UnresolvedAttribute, ...)`). The + // schema is otherwise unused -- consumers read `info` directly and don't iterate `output`. + // SELECT on a view goes through view-text expansion and never produces this node, so giving + // it output does not affect query resolution. + override lazy val output: Seq[Attribute] = + toAttributes(CharVarcharUtils.replaceCharVarcharWithStringInSchema(info.schema)) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala new file mode 100644 index 000000000000..90a08e991c20 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector.catalog + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.catalyst.catalog.CatalogTable + +/** + * A v1 [[CatalogTable]] (representing a session-catalog view) exposed through the v2 + * [[ViewInfo]] surface, mirroring the way [[V1Table]] exposes a v1 table CatalogTable through + * the v2 [[Table]] surface. Holds the original [[CatalogTable]] in [[v1Table]] for v1-only + * paths that need the full v1 metadata representation (e.g. `DescribeTableCommand`, + * `ShowCreateTableCommand`, anything that calls `CatalogTable#toLinkedHashMap`). + */ +private[sql] class V1ViewInfo(val v1Table: CatalogTable) + extends ViewInfo(V1ViewInfo.builderFrom(v1Table)) + +private[sql] object V1ViewInfo { + /** + * Convert a v1 [[CatalogTable]] view into a [[ViewInfo.Builder]] with the same fields. + * Used as the {@code super(builder)} argument when constructing a [[V1ViewInfo]]. + */ + private def builderFrom(v1Table: CatalogTable): ViewInfo.Builder = { + val builder = new ViewInfo.Builder() + builder.withSchema(v1Table.schema) + builder.withProperties(v1Table.properties.asJava) + // ViewInfo requires a non-null queryText; v1 views always have one, but defend against + // an old/corrupt CatalogTable with `viewText = None` by falling back to an empty string. + builder.withQueryText(v1Table.viewText.getOrElse("")) + val cn = v1Table.viewCatalogAndNamespace + if (cn.nonEmpty) { + builder.withCurrentCatalog(cn.head) + builder.withCurrentNamespace(cn.tail.toArray) + } + builder.withSqlConfigs(v1Table.viewSQLConfigs.asJava) + Option(v1Table.viewSchemaMode).foreach(m => builder.withSchemaMode(m.toString)) + builder.withQueryColumnNames(v1Table.viewQueryColumnNames.toArray) + builder + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala index 9a5aed333a4c..40c40f6ea78a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala @@ -272,12 +272,12 @@ class Catalog(sparkSession: SparkSession) extends catalog.Catalog with Logging { if (isExternal) CatalogTableType.EXTERNAL.name else CatalogTableType.MANAGED.name, isTemporary = false) - case ResolvedPersistentView(catalog, identifier, metadata) => + case ResolvedPersistentView(catalog, identifier, info) => new Table( name = identifier.name(), catalog = catalog.name(), namespace = identifier.namespace(), - description = metadata.comment.orNull, + description = info.properties().get(TableCatalog.PROP_COMMENT), tableType = "VIEW", isTemporary = false ) @@ -461,8 +461,8 @@ class Catalog(sparkSession: SparkSession) extends catalog.Catalog with Logging { schemaToColumns(schema, partitionColumnNames.contains, bucketColumnNames.contains, clusteringColumnNames.contains) - case ResolvedPersistentView(_, _, metadata) => - schemaToColumns(metadata.schema) + case ResolvedPersistentView(_, _, info) => + schemaToColumns(info.schema) case ResolvedTempView(_, metadata) => schemaToColumns(metadata.schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala index ed248ccca67a..3ca07f9f79aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala @@ -33,8 +33,8 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.classic.ClassicConversions.castToImpl +import org.apache.spark.sql.connector.catalog.{V1Table, V1ViewInfo} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ -import org.apache.spark.sql.connector.catalog.V1Table import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types._ import org.apache.spark.sql.util.PartitioningUtils @@ -71,10 +71,13 @@ case class DescribeRelationJsonCommand( if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(v.identifier.name()) } + // v1 paths reach here only for session-catalog views (`V1ViewInfo`); v2 views go + // through `DescribeV2ViewExec` in the v2 strategy. + val metadata = v.info.asInstanceOf[V1ViewInfo].v1Table describeIdentifier(v.identifier.toQualifiedNameParts(v.catalog), jsonMap) - describeColsJson(v.metadata.schema, jsonMap) - describeFormattedTableInfoJson(v.metadata, jsonMap) - describeViewSqlConfsJson(v.metadata, jsonMap) + describeColsJson(metadata.schema, jsonMap) + describeFormattedTableInfoJson(metadata, jsonMap) + describeViewSqlConfsJson(metadata, jsonMap) case ResolvedTable(catalog, identifier, V1Table(metadata), _) => describeIdentifier(identifier.toQualifiedNameParts(catalog), jsonMap) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index c98b124b09ff..fb2dc0a68494 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIfNeeded, CaseInsensitiveMap, CharVarcharUtils, DateTimeUtils, ResolveDefaultColumns} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY import org.apache.spark.sql.classic.ClassicConversions.castToImpl -import org.apache.spark.sql.connector.catalog.{TableCatalog, V1Table} +import org.apache.spark.sql.connector.catalog.{TableCatalog, V1Table, V1ViewInfo} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.TableIdentifierHelper import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.CommandExecutionMode @@ -590,7 +590,9 @@ object ResolvedChildHelper { val catalog = sparkSession.sessionState.catalog child match { case ResolvedTempView(_, metadata) => metadata - case ResolvedPersistentView(_, _, metadata) => metadata + // v1 inspection commands always see a v1 (`V1ViewInfo`) view here -- the v2 strategy + // handles non-session views before this method is reached. + case ResolvedPersistentView(_, _, info: V1ViewInfo) => info.v1Table case ResolvedTable(_, _, t: V1Table, _) => t.v1Table case _ if (catalog.isTempView(table)) => catalog.getTempViewOrPermanentTableMetadata(table) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala index cb21e773d86c..3938c08b0179 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala @@ -20,45 +20,50 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{NoSuchViewException, ResolvedIdentifier, ViewSchemaMode} +import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ViewSchemaMode} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, ViewCatalog, ViewInfo} -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper -import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command.CommandUtils +private[v2] object V2ViewMetadataMutation { + /** + * Construct a [[ViewInfo.Builder]] seeded from an existing view's metadata. Mutating execs + * (SET / UNSET TBLPROPERTIES, ALTER VIEW ... WITH SCHEMA BINDING) start here, override the + * one field they're changing, and call [[ViewInfo.Builder#build]] to produce the replacement + * payload for [[ViewCatalog#replaceView]]. Everything else -- columns, queryText, captured + * resolution context, captured SQL configs, queryColumnNames -- flows through unchanged so + * a metadata-only mutation does not perturb the view body. + */ + def builderFrom(existing: ViewInfo): ViewInfo.Builder = { + val builder = new ViewInfo.Builder() + builder + .withSchema(existing.schema) + .withProperties(existing.properties) + .withQueryText(existing.queryText) + .withSqlConfigs(existing.sqlConfigs) + .withCurrentNamespace(existing.currentNamespace) + .withQueryColumnNames(existing.queryColumnNames) + Option(existing.currentCatalog).foreach(builder.withCurrentCatalog) + Option(existing.schemaMode).foreach(builder.withSchemaMode) + builder + } +} + /** - * Shared bits for the v2 ALTER VIEW ... AS exec. Loads the existing view once via - * `existingView` and uses it to preserve user-set TBLPROPERTIES, comment, collation, owner, - * and schema binding mode when constructing the replacement [[ViewInfo]]. A racing DDL between - * analysis and exec can change the target out from under us (dropped, or replaced with a - * non-view table); in that case we surface a regular no-such-view / not-a-view analysis error - * rather than propagating a stale analyzer decision. + * Shared bits for the v2 ALTER VIEW ... AS exec. The replacement [[ViewInfo]] is constructed by + * [[V2ViewPreparation.buildViewInfo]]; the existing view's payload is provided at analysis time + * via the `existingView` field so we can preserve user-set TBLPROPERTIES, comment, collation, + * owner, and schema binding mode without re-loading at runtime. * - * Transient fields (SQL configs, query column names) are re-captured from the - * current session by [[V2ViewPreparation.buildViewInfo]], matching v1 - * `AlterViewAsCommand.alterPermanentView`. PROP_OWNER and user TBLPROPERTIES flow through - * unchanged. + * Transient fields (SQL configs, query column names) are re-captured from the current session, + * matching v1 `AlterViewAsCommand.alterPermanentView`. PROP_OWNER and user TBLPROPERTIES flow + * through unchanged. If the view has been dropped or replaced with a non-view table between + * analysis and exec, the catalog's `replaceView` surfaces `NoSuchViewException` and the error + * propagates. */ private[v2] trait V2AlterViewPreparation extends V2ViewPreparation { - protected lazy val existingView: ViewInfo = try { - catalog.loadView(identifier) - } catch { - case _: NoSuchViewException => - // Race: the view disappeared after analysis. Surface no-such-view, or - // expect-view-not-table if a colliding non-view table appeared in a mixed catalog. - catalog match { - case tc: TableCatalog if tc.tableExists(identifier) => - throw QueryCompilationErrors.expectViewNotTableError( - (catalog.name() +: identifier.asMultipartIdentifier).toSeq, - cmd = "ALTER VIEW ... AS", - suggestAlternative = false, - t = this) - case _ => - throw new NoSuchViewException(identifier) - } - } + protected def existingView: ViewInfo protected lazy val existingProps: Map[String, String] = existingView.properties.asScala.toMap @@ -84,14 +89,6 @@ private[v2] trait V2AlterViewPreparation extends V2ViewPreparation { Option(existingView.schemaMode) .map(CatalogTable.VIEW_SCHEMA_MODE -> _) .toMap) - - /** - * Force-evaluate `existingView` so `NoSuchViewException` / `expectViewNotTableError` - * surfaces before any other work (e.g. `buildViewInfo`, uncache, replace). The result is - * intentionally discarded; call this purely for its side effect of materializing the - * lazy val. - */ - protected def requireExistingView(): Unit = existingView } /** @@ -101,11 +98,11 @@ private[v2] trait V2AlterViewPreparation extends V2ViewPreparation { case class AlterV2ViewExec( catalog: ViewCatalog, identifier: Identifier, + existingView: ViewInfo, originalText: String, query: LogicalPlan) extends V2AlterViewPreparation { override protected def run(): Seq[InternalRow] = { - requireExistingView() val info = buildViewInfo() // Cyclic reference detection is done at analysis time in CheckViewReferences. CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) @@ -113,3 +110,101 @@ case class AlterV2ViewExec( Seq.empty } } + +/** + * Physical plan node for ALTER VIEW ... SET TBLPROPERTIES on a v2 [[ViewCatalog]]. Merges the + * user-supplied properties on top of the analysis-time view properties and dispatches to + * [[ViewCatalog#replaceView]] -- views carry no data, so a single atomic-swap call is sufficient. + */ +case class AlterV2ViewSetPropertiesExec( + catalog: ViewCatalog, + identifier: Identifier, + existingView: ViewInfo, + properties: Map[String, String]) extends LeafV2CommandExec { + + override def output: Seq[org.apache.spark.sql.catalyst.expressions.Attribute] = Seq.empty + + override protected def run(): Seq[InternalRow] = { + val merged = existingView.properties.asScala.toMap ++ properties + val info = V2ViewMetadataMutation.builderFrom(existingView) + .withProperties(merged.asJava) + .build() + catalog.replaceView(identifier, info) + Seq.empty + } +} + +/** + * Physical plan node for ALTER VIEW ... UNSET TBLPROPERTIES on a v2 [[ViewCatalog]]. Drops the + * listed property keys from the analysis-time view properties and dispatches to + * [[ViewCatalog#replaceView]]. Missing keys are silently dropped, matching v1 + * `AlterTableUnsetPropertiesCommand` for views (`ifExists` is unused on the view path -- the + * v1 view command never errors on missing keys). + */ +case class AlterV2ViewUnsetPropertiesExec( + catalog: ViewCatalog, + identifier: Identifier, + existingView: ViewInfo, + propertyKeys: Seq[String]) extends LeafV2CommandExec { + + override def output: Seq[org.apache.spark.sql.catalyst.expressions.Attribute] = Seq.empty + + override protected def run(): Seq[InternalRow] = { + val remaining = existingView.properties.asScala.toMap -- propertyKeys + val info = V2ViewMetadataMutation.builderFrom(existingView) + .withProperties(remaining.asJava) + .build() + catalog.replaceView(identifier, info) + Seq.empty + } +} + +/** + * Physical plan node for ALTER VIEW ... WITH SCHEMA BINDING on a v2 [[ViewCatalog]]. Replaces + * the schema-binding mode on the analysis-time view payload and dispatches to + * [[ViewCatalog#replaceView]]. The view body itself is not re-analyzed -- only the binding mode + * field changes. + */ +case class AlterV2ViewSchemaBindingExec( + catalog: ViewCatalog, + identifier: Identifier, + existingView: ViewInfo, + viewSchemaMode: ViewSchemaMode) extends LeafV2CommandExec { + + override def output: Seq[org.apache.spark.sql.catalyst.expressions.Attribute] = Seq.empty + + override protected def run(): Seq[InternalRow] = { + val info = V2ViewMetadataMutation.builderFrom(existingView) + .withSchemaMode(viewSchemaMode.toString) + .build() + CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) + catalog.replaceView(identifier, info) + Seq.empty + } +} + +/** + * Physical plan node for ALTER VIEW ... RENAME TO on a v2 [[ViewCatalog]]. Dispatches to + * [[ViewCatalog#renameView]]; if the source view is missing or has been replaced with a non-view + * table between analysis and exec, the catalog throws `NoSuchViewException` and the error + * propagates. + */ +case class RenameV2ViewExec( + catalog: ViewCatalog, + oldIdent: Identifier, + newIdent: Identifier) extends LeafV2CommandExec { + + override def output: Seq[org.apache.spark.sql.catalyst.expressions.Attribute] = Seq.empty + + override protected def run(): Seq[InternalRow] = { + // If the new identifier consists of a name only, rename in place within the source + // namespace -- matches `RenameTableExec`'s v1-parity behavior. + val qualifiedNewIdent = if (newIdent.namespace.isEmpty) { + Identifier.of(oldIdent.namespace, newIdent.name) + } else newIdent + CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, oldIdent)) + catalog.invalidateView(oldIdent) + catalog.renameView(oldIdent, qualifiedNewIdent) + Seq.empty + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index e03928867e24..e02059d89692 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkIllegalArgumentException} import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EXPR -import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedPersistentView, ResolvedTable, ResolvedTempView} +import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedPersistentView, ResolvedTable, ResolvedTempView, UnresolvedAttribute} import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning, Expression, NamedExpression, Not, Or, PredicateHelper, SubqueryExpression} @@ -32,9 +32,9 @@ import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreePattern.SCALAR_SUBQUERY -import org.apache.spark.sql.catalyst.util.{toPrettySQL, GeneratedColumn, IdentityColumn, ResolveDefaultColumns, ResolveTableConstraints, V2ExpressionBuilder} +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, GeneratedColumn, IdentityColumn, ResolveDefaultColumns, ResolveTableConstraints, V2ExpressionBuilder} import org.apache.spark.sql.classic.SparkSession -import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, TableCapability, TableCatalog, TruncatableTable, V1Table, ViewCatalog} +import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, TableCapability, TableCatalog, TruncatableTable, V1Table, V1ViewInfo, ViewCatalog} import org.apache.spark.sql.connector.catalog.TableChange import org.apache.spark.sql.connector.catalog.index.SupportsIndex import org.apache.spark.sql.connector.expressions.{FieldReference, LiteralValue} @@ -103,6 +103,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat location, session.sharedState.hadoopConf) } + // Strategy cases that target v2 views read `ResolvedPersistentView.info` directly. For + // session-catalog (v1) views the payload is a `V1ViewInfo` wrapping the original + // `CatalogTable`; v2 catalogs supply a regular `ViewInfo` from the catalog. + // `ResolveSessionCatalog` rewrites session-catalog views to v1 commands before this strategy + // fires, so v2 cases that don't expect a `V1ViewInfo` won't see one. + private def qualifyLocInTableSpec(tableSpec: TableSpec): TableSpec = { val newLoc = tableSpec.location.map { loc => val locationUri = CatalogUtils.stringToURI(loc) @@ -265,7 +271,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat locationStr, provider, serdeInfo, properties, ifNotExists) => val table = source match { case ResolvedTable(_, _, t, _) => t - case ResolvedPersistentView(_, _, meta) => V1Table(meta) + case ResolvedPersistentView(_, _, info: V1ViewInfo) => V1Table(info.v1Table) case ResolvedTempView(_, meta) => V1Table(meta) } val location = locationStr.map { loc => @@ -312,49 +318,50 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat CreateV2ViewExec(catalog.asInstanceOf[ViewCatalog], ident, userSpecifiedColumns, comment, collation, properties, sqlText, child, allowExisting, replace, viewSchemaMode) :: Nil - case AlterViewAs(ResolvedPersistentView(catalog, ident, _), originalText, query, _, _) => - AlterV2ViewExec(catalog.asInstanceOf[ViewCatalog], ident, originalText, query) :: Nil + case AlterViewAs(rpv @ ResolvedPersistentView(catalog, ident, _), + originalText, query, _, _) => + AlterV2ViewExec(catalog.asInstanceOf[ViewCatalog], ident, rpv.info, + originalText, query) :: Nil // View DDL / inspection on a non-session v2 catalog that the v1 rewrite in - // `ResolveSessionCatalog` can't handle. These are tracked as follow-up work in SPARK-52729; - // pin the current failure mode with a clean `UNSUPPORTED_FEATURE.TABLE_OPERATION` error - // so users get a meaningful message (and test coverage catches a future regression to a - // generic planner error). - case SetViewProperties(ResolvedPersistentView(catalog, ident, _), _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "ALTER VIEW ... SET TBLPROPERTIES") - - case UnsetViewProperties(ResolvedPersistentView(catalog, ident, _), _, _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "ALTER VIEW ... UNSET TBLPROPERTIES") - - case AlterViewSchemaBinding(ResolvedPersistentView(catalog, ident, _), _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "ALTER VIEW ... WITH SCHEMA") - - case RenameTable(ResolvedPersistentView(catalog, ident, _), _, _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "ALTER VIEW ... RENAME TO") - - case ShowCreateTable(ResolvedPersistentView(catalog, ident, _), _, _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "SHOW CREATE TABLE") - - case ShowTableProperties(ResolvedPersistentView(catalog, ident, _), _, _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "SHOW TBLPROPERTIES") - - case ShowColumns(ResolvedPersistentView(catalog, ident, _), _, _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "SHOW COLUMNS") - - case DescribeRelation(ResolvedPersistentView(catalog, ident, _), _, _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "DESCRIBE TABLE") - - case DescribeColumn(ResolvedPersistentView(catalog, ident, _), _, _, _) => - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "DESCRIBE TABLE ... COLUMN") + // `ResolveSessionCatalog` can't handle (its `ResolvedViewIdentifier` matcher is gated on + // `isSessionCatalog`). Routed to dedicated v2 execs that read the typed `ViewInfo` + // resolved at analysis time directly from `ResolvedPersistentView.viewInfo` -- no + // re-loading at exec time. + case SetViewProperties(rpv @ ResolvedPersistentView(catalog, ident, _), props) => + AlterV2ViewSetPropertiesExec( + catalog.asInstanceOf[ViewCatalog], ident, rpv.info, props) :: Nil + + case UnsetViewProperties(rpv @ ResolvedPersistentView(catalog, ident, _), keys, _) => + AlterV2ViewUnsetPropertiesExec( + catalog.asInstanceOf[ViewCatalog], ident, rpv.info, keys) :: Nil + + case AlterViewSchemaBinding(rpv @ ResolvedPersistentView(catalog, ident, _), schemaMode) => + AlterV2ViewSchemaBindingExec( + catalog.asInstanceOf[ViewCatalog], ident, rpv.info, schemaMode) :: Nil + + case RenameTable(ResolvedPersistentView(catalog, ident, _), newName, _) => + RenameV2ViewExec( + catalog.asInstanceOf[ViewCatalog], ident, newName.asIdentifier) :: Nil + + case ShowCreateTable(rpv @ ResolvedPersistentView(catalog, ident, _), _, output) => + val quoted = (catalog.name() +: ident.asMultipartIdentifier).map(quoteIfNeeded).mkString(".") + ShowCreateV2ViewExec(output, quoted, rpv.info) :: Nil + + case ShowTableProperties(rpv @ ResolvedPersistentView(catalog, ident, _), + propertyKey, output) => + val quoted = (catalog.name() +: ident.asMultipartIdentifier).map(quoteIfNeeded).mkString(".") + ShowV2ViewPropertiesExec(output, quoted, rpv.info, propertyKey) :: Nil + + case ShowColumns(rpv @ ResolvedPersistentView(_, _, _), _, output) => + ShowV2ViewColumnsExec(output, rpv.info) :: Nil + + case DescribeRelation(rpv @ ResolvedPersistentView(catalog, ident, _), isExtended, output) => + DescribeV2ViewExec(output, catalog.name(), ident, rpv.info, isExtended) :: Nil + + case DescribeColumn(rpv @ ResolvedPersistentView(_, _, _), column: UnresolvedAttribute, + isExtended, output) => + DescribeV2ViewColumnExec(output, rpv.info, column, isExtended) :: Nil // Plans that resolve through `UnresolvedTableOrView` reach here with a // `ResolvedPersistentView` child for non-session v2 views (the v1 rewrite in diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 8ccd126b4b22..38140d62cb15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -22,12 +22,18 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.StringUtils -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog, TableViewCatalog} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.execution.LeafExecNode /** * Physical plan node for showing tables. + * + * For a [[TableViewCatalog]] (one that exposes both tables and views in a shared identifier + * namespace), this routes through [[TableViewCatalog#listRelationSummaries]] so that views are + * included in the listing -- matching the v1 `SHOW TABLES` semantics where views appear + * alongside tables. Pure [[TableCatalog]] catalogs continue to use `listTables` and return + * tables only. */ case class ShowTablesExec( output: Seq[Attribute], @@ -37,10 +43,13 @@ case class ShowTablesExec( override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - val tables = catalog.listTables(namespace.toArray) - tables.map { table => - if (pattern.map(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty).getOrElse(true)) { - rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table, catalog)) + val identifiers: Array[Identifier] = catalog match { + case mc: TableViewCatalog => mc.listRelationSummaries(namespace.toArray).map(_.identifier()) + case _ => catalog.listTables(namespace.toArray) + } + identifiers.foreach { ident => + if (pattern.map(StringUtils.filterPattern(Seq(ident.name()), _).nonEmpty).getOrElse(true)) { + rows += toCatalystRow(ident.namespace().quoted, ident.name(), isTempView(ident, catalog)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala new file mode 100644 index 000000000000..41831fbe2cce --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIfNeeded, ResolveDefaultColumns} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumnsUtils.CURRENT_DEFAULT_COLUMN_METADATA_KEY +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, ViewInfo} +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * Read-side v2 view execs. Each receives the typed [[ViewInfo]] resolved at analysis time + * (carried on `ResolvedPersistentView.viewInfo`) and formats output rows directly from it -- + * matching the way v2 table inspection execs (e.g. `ShowCreateTableExec`, `DescribeTableExec`) + * consume the [[org.apache.spark.sql.connector.catalog.Table]] attached to `ResolvedTable`. + * + * Only non-session v2 views land here; the session-catalog path is rewritten to v1 commands by + * `ResolveSessionCatalog` before strategy fires. The catalog name and identifier are passed + * alongside `viewInfo` for output formatting (qualified names, EXTENDED block headers). + */ + +/** + * Physical plan node for SHOW CREATE TABLE on a v2 view. Reconstructs the {@code CREATE VIEW} + * statement directly from the typed [[ViewInfo]] -- the column list comes from + * [[ViewInfo#schema]], the body from [[ViewInfo#queryText]], the binding mode from + * [[ViewInfo#schemaMode]], and the user TBLPROPERTIES from [[ViewInfo#properties]] (with the + * reserved-keys filter applied so internal entries don't leak into the rendered DDL). + */ +case class ShowCreateV2ViewExec( + output: Seq[Attribute], + quotedName: String, + viewInfo: ViewInfo) extends LeafV2CommandExec with SQLConfHelper { + + override protected def run(): Seq[InternalRow] = { + val builder = new StringBuilder + builder ++= s"CREATE VIEW $quotedName " + showViewDataColumns(builder) + Option(viewInfo.properties.get(TableCatalog.PROP_COMMENT)).foreach { c => + builder ++= s"COMMENT '${escapeSingleQuotedString(c)}'\n" + } + Option(viewInfo.properties.get(TableCatalog.PROP_COLLATION)).foreach { c => + builder ++= s"DEFAULT COLLATION $c\n" + } + showViewProperties(builder) + if (conf.viewSchemaBindingEnabled) { + Option(viewInfo.schemaMode).foreach { sm => + builder ++= s"WITH SCHEMA $sm\n" + } + } + builder ++= s"AS ${viewInfo.queryText}\n" + Seq(toCatalystRow(builder.toString)) + } + + private def showViewDataColumns(builder: StringBuilder): Unit = { + val schema = viewInfo.schema + if (schema.nonEmpty) { + val cols = schema.map { f => + val comment = f.getComment().map(c => s" COMMENT '${escapeSingleQuotedString(c)}'") + s"${quoteIfNeeded(f.name)}${comment.getOrElse("")}" + } + builder ++= cols.mkString("(\n ", ",\n ", ")\n") + } + } + + private def showViewProperties(builder: StringBuilder): Unit = { + // Drop the reserved keys that already appear as dedicated DDL clauses (PROP_COMMENT / + // PROP_COLLATION) and the auto-injected `table_type`. Anything left is user TBLPROPERTIES. + val reserved = Set( + TableCatalog.PROP_COMMENT, + TableCatalog.PROP_COLLATION, + TableCatalog.PROP_TABLE_TYPE) + val viewProps = viewInfo.properties.asScala + .filter { case (k, _) => !reserved.contains(k) } + if (viewProps.nonEmpty) { + val props = viewProps.toSeq.sortBy(_._1).map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + builder ++= s"TBLPROPERTIES ${props.mkString("(\n ", ",\n ", ")\n")}" + } + } +} + +/** + * Physical plan node for SHOW TBLPROPERTIES on a v2 view. Returns the user-facing properties + * from [[ViewInfo#properties]] -- the auto-injected `table_type` is filtered out so users see + * only what they (or the catalog) explicitly set. + */ +case class ShowV2ViewPropertiesExec( + output: Seq[Attribute], + quotedName: String, + viewInfo: ViewInfo, + propertyKey: Option[String]) extends LeafV2CommandExec with SQLConfHelper { + + override protected def run(): Seq[InternalRow] = { + val rawProps = viewInfo.properties.asScala.toMap + val redacted = conf.redactOptions(rawProps) + propertyKey match { + case Some(p) => + val propValue = redacted.getOrElse(p, + s"View $quotedName does not have property: $p") + if (output.length == 1) { + Seq(toCatalystRow(propValue)) + } else { + Seq(toCatalystRow(p, propValue)) + } + case None => + redacted + .filter { case (k, _) => k != TableCatalog.PROP_TABLE_TYPE } + .toSeq.sortBy(_._1).map(p => toCatalystRow(p._1, p._2)) + } + } +} + +/** + * Physical plan node for SHOW COLUMNS on a v2 view. Returns one row per top-level field in + * [[ViewInfo#schema]]. + */ +case class ShowV2ViewColumnsExec( + output: Seq[Attribute], + viewInfo: ViewInfo) extends LeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + viewInfo.schema.map(c => toCatalystRow(c.name)).toSeq + } +} + +/** + * Physical plan node for DESCRIBE TABLE on a v2 view. Schema rows first; when EXTENDED is + * specified, an additional `# Detailed View Information` block emits the v2-native fields + * (catalog, identifier, view text, captured creation context, schema-binding mode, query + * column names, user TBLPROPERTIES). v2 views are unpartitioned by definition, so the + * partition-spec branch from v1 `DescribeTableCommand` is unreachable here. + */ +case class DescribeV2ViewExec( + output: Seq[Attribute], + catalogName: String, + identifier: Identifier, + viewInfo: ViewInfo, + isExtended: Boolean) extends LeafV2CommandExec with SQLConfHelper { + + override protected def run(): Seq[InternalRow] = { + val result = new ArrayBuffer[InternalRow] + viewInfo.schema.foreach { col => + result += toCatalystRow(col.name, col.dataType.simpleString, col.getComment().orNull) + } + if (isExtended) { + result += toCatalystRow("", "", "") + result += toCatalystRow("# Detailed View Information", "", "") + result += toCatalystRow("Catalog", catalogName, "") + val qualified = (identifier.namespace() :+ identifier.name()) + .map(quoteIfNeeded).mkString(".") + result += toCatalystRow("Identifier", qualified, "") + result += toCatalystRow("View Text", viewInfo.queryText, "") + Option(viewInfo.currentCatalog).foreach { c => + result += toCatalystRow("View Current Catalog", c, "") + } + val ns = viewInfo.currentNamespace + if (ns != null && ns.nonEmpty) { + result += toCatalystRow( + "View Current Namespace", ns.map(quoteIfNeeded).mkString("."), "") + } + Option(viewInfo.schemaMode).foreach { sm => + result += toCatalystRow("View Schema Mode", sm, "") + } + val queryColumns = viewInfo.queryColumnNames + if (queryColumns != null && queryColumns.nonEmpty) { + result += toCatalystRow( + "View Query Output Columns", queryColumns.mkString("[", ", ", "]"), "") + } + val userProps = viewInfo.properties.asScala + .filter { case (k, _) => k != TableCatalog.PROP_TABLE_TYPE } + if (userProps.nonEmpty) { + val props = conf.redactOptions(userProps.toMap).toSeq.sortBy(_._1).map { + case (k, v) => s"$k=$v" + }.mkString("[", ", ", "]") + result += toCatalystRow("Properties", props, "") + } + } + ResolveDefaultColumns.getDescribeMetadata(viewInfo.schema).foreach { row => + result += toCatalystRow(row._1, row._2, row._3) + } + result.toSeq + } +} + +/** + * Physical plan node for DESCRIBE TABLE ... COLUMN on a v2 view. Mirrors the column-only + * branch of v1 `DescribeColumnCommand`: emit `col_name`, `data_type`, `comment` for the + * resolved field. v2 views do not carry column statistics, so the EXTENDED branch in v1 emits + * `NULL` for every stat row -- we follow the same shape. + */ +case class DescribeV2ViewColumnExec( + output: Seq[Attribute], + viewInfo: ViewInfo, + column: UnresolvedAttribute, + isExtended: Boolean) extends LeafV2CommandExec with SQLConfHelper { + + override protected def run(): Seq[InternalRow] = { + val resolver = conf.resolver + val colNameParts = column.nameParts + val colName = column.name + if (colNameParts.length > 1) { + throw QueryCompilationErrors.commandNotSupportNestedColumnError( + "DESC TABLE COLUMN", colName) + } + val field = viewInfo.schema.fields + .find(f => resolver(f.name, colNameParts.head)) + .getOrElse(throw QueryCompilationErrors.columnNotFoundError(colName)) + val dataType = field.dataType.catalogString + val comment = field.getComment().orNull + val rows = ArrayBuffer[InternalRow]( + toCatalystRow("col_name", field.name), + toCatalystRow("data_type", dataType), + toCatalystRow("comment", if (comment == null) "NULL" else comment) + ) + if (isExtended) { + // v2 views carry no column stats; emit NULL placeholders matching v1 output shape. + Seq("min", "max", "num_nulls", "distinct_count", "avg_col_len", "max_col_len", + "histogram").foreach { name => + rows += toCatalystRow(name, "NULL") + } + if (field.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) { + rows += toCatalystRow("default", + field.metadata.getString(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) + } + } + rows.toSeq + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala index a24f71884ed7..38894b75ad92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala @@ -34,11 +34,10 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap * Data-source-table read paths live in * [[org.apache.spark.sql.connector.DataSourceV2MetadataTableSuite]]. * - * TODO: once the remaining v2 view DDL is implemented (SET/UNSET TBLPROPERTIES, SHOW CREATE - * VIEW, RENAME TO, SCHEMA BINDING, DESCRIBE / SHOW TBLPROPERTIES on v2 views), register a - * `MetadataTable`-backed `DelegatingCatalogExtension` as `spark.sql.catalog.spark_catalog` - * and run the shared [[org.apache.spark.sql.execution.PersistedViewTestSuite]] body against - * the v2 path for full parity with the v1 persisted-view coverage. + * TODO: register a `MetadataTable`-backed `DelegatingCatalogExtension` as + * `spark.sql.catalog.spark_catalog` and run the shared + * [[org.apache.spark.sql.execution.PersistedViewTestSuite]] body against the v2 path for full + * parity with the v1 persisted-view coverage. */ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -648,72 +647,85 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { } } - // --- Follow-up-blocked view DDL / inspection on a non-session v2 catalog ------------ - // These plans don't have a dedicated v2 strategy yet (tracked for a follow-up PR). We pin - // the current failure mode -- UNSUPPORTED_FEATURE.TABLE_OPERATION with a statement-specific - // operation string -- so a future generic "no plan found" regression would surface here - // rather than silently degrading the UX. + // --- v2 view DDL / inspection on a non-session v2 catalog ---------------------------- + // ResolveSessionCatalog's `ResolvedViewIdentifier` matcher is gated on isSessionCatalog, so + // these plans flow through to DataSourceV2Strategy with a `ResolvedPersistentView` child. + // Each is handled by a dedicated v2 exec defined alongside the v1 commands. private def seedV2View(name: String): Unit = { sql(s"CREATE VIEW view_catalog.default.$name AS SELECT 1 AS x") } + // Used by the REFRESH / ANALYZE pins below: those plans still don't have a v2 implementation + // and surface UNSUPPORTED_FEATURE.TABLE_OPERATION via DataSourceV2Strategy. private def assertUnsupportedViewOp(statement: String): Unit = { val ex = intercept[AnalysisException](sql(statement)) assert(ex.getCondition == "UNSUPPORTED_FEATURE.TABLE_OPERATION", s"got ${ex.getCondition}") } - test("ALTER VIEW ... SET TBLPROPERTIES on a v2 view is rejected") { + test("ALTER VIEW ... SET TBLPROPERTIES on a v2 view writes the properties") { + val catalog = spark.sessionState.catalogManager.catalog("view_catalog") + .asInstanceOf[TestingTableViewCatalog] seedV2View("v_set_props") - assertUnsupportedViewOp( - "ALTER VIEW view_catalog.default.v_set_props SET TBLPROPERTIES ('k' = 'v')") + sql("ALTER VIEW view_catalog.default.v_set_props SET TBLPROPERTIES ('k' = 'v')") + val stored = catalog.getStoredView(Array("default"), "v_set_props") + assert(stored.properties.get("k") == "v") } - test("ALTER VIEW ... UNSET TBLPROPERTIES on a v2 view is rejected") { + test("ALTER VIEW ... UNSET TBLPROPERTIES on a v2 view drops the properties") { + val catalog = spark.sessionState.catalogManager.catalog("view_catalog") + .asInstanceOf[TestingTableViewCatalog] seedV2View("v_unset_props") - assertUnsupportedViewOp( - "ALTER VIEW view_catalog.default.v_unset_props UNSET TBLPROPERTIES ('k')") + sql("ALTER VIEW view_catalog.default.v_unset_props SET TBLPROPERTIES ('k' = 'v')") + sql("ALTER VIEW view_catalog.default.v_unset_props UNSET TBLPROPERTIES ('k')") + val stored = catalog.getStoredView(Array("default"), "v_unset_props") + assert(!stored.properties.containsKey("k")) } - test("ALTER VIEW ... WITH SCHEMA on a v2 view is rejected") { + test("ALTER VIEW ... WITH SCHEMA on a v2 view updates schemaMode") { + val catalog = spark.sessionState.catalogManager.catalog("view_catalog") + .asInstanceOf[TestingTableViewCatalog] seedV2View("v_schema_binding") - assertUnsupportedViewOp( - "ALTER VIEW view_catalog.default.v_schema_binding WITH SCHEMA EVOLUTION") + sql("ALTER VIEW view_catalog.default.v_schema_binding WITH SCHEMA EVOLUTION") + val stored = catalog.getStoredView(Array("default"), "v_schema_binding") + assert(stored.schemaMode == "EVOLUTION") } - test("ALTER VIEW ... RENAME TO on a v2 view is rejected") { + test("ALTER VIEW ... RENAME TO on a v2 view moves the entry") { + val catalog = spark.sessionState.catalogManager.catalog("view_catalog") + .asInstanceOf[TestingTableViewCatalog] seedV2View("v_rename") - assertUnsupportedViewOp( - "ALTER VIEW view_catalog.default.v_rename RENAME TO view_catalog.default.v_renamed") + sql("ALTER VIEW view_catalog.default.v_rename RENAME TO view_catalog.default.v_renamed") + assert(!catalog.viewExists(Identifier.of(Array("default"), "v_rename"))) + assert(catalog.viewExists(Identifier.of(Array("default"), "v_renamed"))) } - test("SHOW CREATE TABLE on a v2 view is rejected") { + test("SHOW CREATE TABLE on a v2 view emits CREATE VIEW") { seedV2View("v_show_create") - assertUnsupportedViewOp("SHOW CREATE TABLE view_catalog.default.v_show_create") + val ddl = sql("SHOW CREATE TABLE view_catalog.default.v_show_create") + .collect().head.getString(0) + assert(ddl.startsWith("CREATE VIEW ")) + assert(ddl.contains("AS SELECT 1 AS x")) } - test("SHOW TBLPROPERTIES on a v2 view is rejected") { + test("SHOW TBLPROPERTIES on a v2 view returns user properties") { seedV2View("v_show_props") - assertUnsupportedViewOp("SHOW TBLPROPERTIES view_catalog.default.v_show_props") + sql("ALTER VIEW view_catalog.default.v_show_props SET TBLPROPERTIES ('k' = 'v')") + val rows = sql("SHOW TBLPROPERTIES view_catalog.default.v_show_props").collect() + assert(rows.exists(r => r.getString(0) == "k" && r.getString(1) == "v")) } - test("SHOW COLUMNS on a v2 view is rejected") { + test("SHOW COLUMNS on a v2 view returns the column list") { seedV2View("v_show_cols") - assertUnsupportedViewOp("SHOW COLUMNS IN view_catalog.default.v_show_cols") + val cols = sql("SHOW COLUMNS IN view_catalog.default.v_show_cols") + .collect().map(_.getString(0)).toSeq + assert(cols == Seq("x")) } - test("DESCRIBE TABLE on a v2 view is rejected") { + test("DESCRIBE TABLE on a v2 view returns the schema") { seedV2View("v_describe") - assertUnsupportedViewOp("DESCRIBE TABLE view_catalog.default.v_describe") - } - - test("DESCRIBE TABLE ... COLUMN on a v2 view is rejected") { - seedV2View("v_describe_col") - // Column resolution against a v2 view's output isn't wired up yet, so the analyzer fails - // with UNRESOLVED_COLUMN before reaching the planner. That's still a clean - // AnalysisException (not a generic "no plan found"), which is the pin we care about. - intercept[AnalysisException]( - sql("DESCRIBE TABLE view_catalog.default.v_describe_col x")) + val rows = sql("DESCRIBE TABLE view_catalog.default.v_describe").collect() + assert(rows.exists(r => r.getString(0) == "x" && r.getString(1) == "int")) } // These plans reach `DataSourceV2Strategy` with a `ResolvedPersistentView` child on a @@ -808,18 +820,17 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { .build()) } - test("SHOW TABLES on a v2 catalog returns only tables") { - // Per the new `TableCatalog.listTables` contract, SHOW TABLES returns table identifiers - // only -- views (in mixed catalogs) are listed via SHOW VIEWS / `ViewCatalog.listViews`. - // This is an intentional divergence from v1 SHOW TABLES (which includes both tables and - // views in a single listing); v2 catalogs separate the two so callers can target either - // kind without filtering. + test("SHOW TABLES on a TableViewCatalog returns both tables and views (v1-parity)") { + // For a `TableViewCatalog` (a catalog exposing both tables and views in a shared + // identifier namespace), SHOW TABLES routes through `listRelationSummaries` so views + // appear alongside tables -- matching the v1 SHOW TABLES output. Pure `TableCatalog` + // catalogs (no view mixin) continue to use `listTables` and return tables only. seedV2View("v_in_show_tables") seedV2Table("t_in_show_tables") val rows = sql("SHOW TABLES IN view_catalog.default").collect() val names = rows.map(_.getString(1)).toSet assert(names.contains("t_in_show_tables"), s"table missing from SHOW TABLES: $names") - assert(!names.contains("v_in_show_tables"), s"view leaked into SHOW TABLES: $names") + assert(names.contains("v_in_show_tables"), s"view missing from SHOW TABLES: $names") rows.foreach(r => assert(!r.getBoolean(2), s"isTemporary must be false: $r")) } @@ -1025,6 +1036,19 @@ class TestingTableViewCatalog extends TableViewCatalog { createdViews.remove(key) != null } + override def renameView(oldIdent: Identifier, newIdent: Identifier): Unit = { + val oldKey = (oldIdent.namespace().toSeq, oldIdent.name()) + val newKey = (newIdent.namespace().toSeq, newIdent.name()) + val existing = createdViews.get(oldKey) + if (existing == null || !existing.isInstanceOf[ViewInfo]) { + throw new NoSuchViewException(oldIdent) + } + if (createdViews.putIfAbsent(newKey, existing) != null) { + throw new ViewAlreadyExistsException(newIdent) + } + createdViews.remove(oldKey) + } + private var catalogName = "" override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { catalogName = name @@ -1111,6 +1135,17 @@ class TestingViewOnlyCatalog extends ViewCatalog { store.remove(key) != null } + override def renameView(oldIdent: Identifier, newIdent: Identifier): Unit = { + val oldKey = (oldIdent.namespace().toSeq, oldIdent.name()) + val newKey = (newIdent.namespace().toSeq, newIdent.name()) + val existing = store.get(oldKey) + if (existing == null) throw new NoSuchViewException(oldIdent) + if (store.putIfAbsent(newKey, existing) != null) { + throw new ViewAlreadyExistsException(newIdent) + } + store.remove(oldKey) + } + private var catalogName = "" override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { catalogName = name From c121e074e847fe23975c3076674859fd37248140 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 03:26:46 +0000 Subject: [PATCH 03/17] [SPARK-56655][SQL][TESTS] Add per-catalog view command test triplets and fold in late prod tweaks Mirror the DROP TABLE test layout from `sql/core/test/.../command/{,v1/,v2/}` for every v2 view DDL / inspection command added in the parent commit. Each command lands as: - `command/SuiteBase.scala` -- unified tests parameterized by `$catalog` - `command/v1/Suite.scala` -- extends Base + v1 `ViewCommandSuiteBase` (pins `$catalog` to `spark_catalog`, so the unified tests target the session catalog) - `command/v2/Suite.scala` -- extends Base + v2 `ViewCommandSuiteBase` (pins `$catalog` to a fresh `test_view_catalog` backed by a new general-purpose `InMemoryTableViewCatalog` test fixture), plus catalog-state assertions specific to the v2 fixture Triplets cover: CREATE VIEW, ALTER VIEW ... AS, ALTER VIEW SET / UNSET TBLPROPERTIES, ALTER VIEW RENAME TO, ALTER VIEW WITH SCHEMA, SHOW CREATE TABLE, SHOW TBLPROPERTIES, SHOW COLUMNS, DESCRIBE TABLE, DESCRIBE TABLE ... COLUMN, DROP VIEW, SHOW VIEWS. Each Base test runs against both `spark_catalog` (v1, hits the existing v1 commands) and `test_view_catalog` (v2, hits the new execs from the parent commit), giving a single source of cross-catalog behavioral parity. Supporting infrastructure: - `sql/catalyst/test/.../connector/catalog/InMemoryTableViewCatalog.scala`: a general-purpose `TableViewCatalog` for tests, with shared table/view keyspace and a runtime-type kind discriminator. - `command/v1/ViewCommandSuiteBase.scala` and `command/v2/ViewCommandSuiteBase.scala`: extend the existing v1/v2 `CommandSuiteBase` so view tests inherit `checkLocation` etc. Folded in here as well (chronologically these landed alongside the test work and depend on the test infrastructure to exercise): - Reject `ALTER TABLE RENAME TO ...` with EXPECT_TABLE_NOT_VIEW .USE_ALTER_VIEW. The parser routes both `ALTER TABLE ... RENAME TO` and `ALTER VIEW ... RENAME TO` to the same `RenameTable` plan with an `isView` flag; the v2 strategy now consults the flag and rejects the wrong-syntax case (mirrors v1 `DDLUtils.verifyAlterTableType`). - DESCRIBE TABLE ... COLUMN on a v2 view: surface the view's schema as `ResolvedPersistentView.output` (with char/varchar normalization). `ResolveReferences` resolves the column reference against it -- analogous to how `ResolvedTable.output` lets the column resolve naturally for tables. `CheckAnalysis` no longer trips, the plan stays as `DescribeColumn(ResolvedPersistentView, ...)` until strategy time, and the v2 case extracts `nameParts` from the resolved attribute and dispatches to a new `DescribeV2ViewColumnExec`. The v1 rewrite in `ResolveSessionCatalog` accepts both the resolved `Attribute` form and the legacy `UnresolvedAttribute` form. The pre-existing `DataSourceV2MetadataViewSuite` is trimmed: CREATE / ALTER / DROP / SHOW VIEW DDL coverage moves to the per-catalog triplets. What remains in the leaf suite is genuinely v2-specific structural coverage (view read-path, V1Table.toCatalogTable round-trip, pure-ViewCatalog read + ALTER, multi-level- namespace cyclic detection / error rendering, REFRESH / ANALYZE rejection, SHOW TABLES on a TableViewCatalog returning both kinds). 220 view-related tests pass locally across 29 suites. Co-authored-by: Isaac --- .../catalog/InMemoryTableViewCatalog.scala | 164 ++++++ .../analysis/ResolveSessionCatalog.scala | 20 +- .../datasources/v2/DataSourceV2Strategy.scala | 27 +- .../v2/V2ViewInspectionExecs.scala | 15 +- .../DataSourceV2MetadataViewSuite.scala | 542 +----------------- .../command/AlterViewAsSuiteBase.scala | 151 +++++ .../command/AlterViewRenameSuiteBase.scala | 86 +++ .../AlterViewSchemaBindingSuiteBase.scala | 70 +++ .../AlterViewSetTblPropertiesSuiteBase.scala | 99 ++++ ...AlterViewUnsetTblPropertiesSuiteBase.scala | 76 +++ .../command/CreateViewSuiteBase.scala | 159 +++++ .../command/DescribeViewColumnSuiteBase.scala | 49 ++ .../command/DescribeViewSuiteBase.scala | 50 ++ .../execution/command/DropViewSuiteBase.scala | 56 ++ .../command/ShowCreateViewSuiteBase.scala | 53 ++ .../command/ShowViewColumnsSuiteBase.scala | 43 ++ .../command/ShowViewPropertiesSuiteBase.scala | 58 ++ .../command/ShowViewsSuiteBase.scala | 47 ++ .../command/v1/AlterViewAsSuite.scala | 22 + .../command/v1/AlterViewRenameSuite.scala | 23 + .../v1/AlterViewSchemaBindingSuite.scala | 23 + .../v1/AlterViewSetTblPropertiesSuite.scala | 27 + .../v1/AlterViewUnsetTblPropertiesSuite.scala | 23 + .../command/v1/CreateViewSuite.scala | 22 + .../command/v1/DescribeViewColumnSuite.scala | 23 + .../command/v1/DescribeViewSuite.scala | 23 + .../execution/command/v1/DropViewSuite.scala | 22 + .../command/v1/ShowCreateViewSuite.scala | 23 + .../command/v1/ShowViewColumnsSuite.scala | 23 + .../command/v1/ShowViewPropertiesSuite.scala | 23 + .../execution/command/v1/ShowViewsSuite.scala | 22 + .../command/v1/ViewCommandSuiteBase.scala | 24 + .../command/v2/AlterViewAsSuite.scala | 61 ++ .../command/v2/AlterViewRenameSuite.scala | 33 ++ .../v2/AlterViewSchemaBindingSuite.scala | 32 ++ .../v2/AlterViewSetTblPropertiesSuite.scala | 36 ++ .../v2/AlterViewUnsetTblPropertiesSuite.scala | 32 ++ .../command/v2/CreateViewSuite.scala | 86 +++ .../command/v2/DescribeViewColumnSuite.scala | 23 + .../command/v2/DescribeViewSuite.scala | 41 ++ .../execution/command/v2/DropViewSuite.scala | 68 +++ .../command/v2/ShowCreateViewSuite.scala | 23 + .../command/v2/ShowViewColumnsSuite.scala | 23 + .../command/v2/ShowViewPropertiesSuite.scala | 23 + .../execution/command/v2/ShowViewsSuite.scala | 58 ++ .../command/v2/ViewCommandSuiteBase.scala | 38 ++ 46 files changed, 2116 insertions(+), 549 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewAsSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSchemaBindingSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewUnsetTblPropertiesSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewColumnSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewPropertiesSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewAsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewRenameSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSchemaBindingSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSetTblPropertiesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewUnsetTblPropertiesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewColumnSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewColumnsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewPropertiesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ViewCommandSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSetTblPropertiesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewUnsetTblPropertiesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewColumnSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateViewSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewColumnsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewPropertiesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ViewCommandSuiteBase.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala new file mode 100644 index 000000000000..28355ba7abe4 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.catalog + +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, NoSuchViewException, TableAlreadyExistsException, ViewAlreadyExistsException} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** + * An in-memory [[TableViewCatalog]] for tests. Tables and views share a single keyspace per + * the [[TableViewCatalog]] contract; the stored value's runtime type ([[TableInfo]] vs + * [[ViewInfo]]) is the kind discriminator. Suitable for any test suite that wants to exercise + * v2 view DDL or inspection commands against a non-session catalog. + */ +class InMemoryTableViewCatalog extends TableViewCatalog { + + private val store = + new ConcurrentHashMap[(Seq[String], String), TableInfo]() + + override def loadTableOrView(ident: Identifier): Table = { + val key = (ident.namespace().toSeq, ident.name()) + Option(store.get(key)) + .map(new MetadataTable(_, ident.toString)) + .getOrElse(throw new NoSuchTableException(ident)) + } + + // ----- TableCatalog ----------------------------------------------------------------- + + override def createTable(ident: Identifier, info: TableInfo): Table = { + val key = (ident.namespace().toSeq, ident.name()) + if (store.putIfAbsent(key, info) != null) { + throw new TableAlreadyExistsException(ident) + } + new MetadataTable(info, ident.toString) + } + + override def alterTable(ident: Identifier, changes: TableChange*): Table = { + throw new UnsupportedOperationException("alterTable not supported on InMemoryTableViewCatalog") + } + + override def dropTable(ident: Identifier): Boolean = { + val key = (ident.namespace().toSeq, ident.name()) + val existing = store.get(key) + if (existing == null || existing.isInstanceOf[ViewInfo]) return false + store.remove(key) != null + } + + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { + val oldKey = (oldIdent.namespace().toSeq, oldIdent.name()) + val newKey = (newIdent.namespace().toSeq, newIdent.name()) + val existing = store.get(oldKey) + if (existing == null || existing.isInstanceOf[ViewInfo]) { + throw new NoSuchTableException(oldIdent) + } + if (store.putIfAbsent(newKey, existing) != null) { + throw new TableAlreadyExistsException(newIdent) + } + store.remove(oldKey) + } + + override def listTables(namespace: Array[String]): Array[Identifier] = { + val target = namespace.toSeq + val ids = new java.util.ArrayList[Identifier]() + store.forEach { (key, info) => + if (key._1 == target && !info.isInstanceOf[ViewInfo]) { + ids.add(Identifier.of(key._1.toArray, key._2)) + } + } + ids.toArray(new Array[Identifier](0)) + } + + // ----- ViewCatalog ------------------------------------------------------------------ + + override def listViews(namespace: Array[String]): Array[Identifier] = { + val target = namespace.toSeq + val ids = new java.util.ArrayList[Identifier]() + store.forEach { (key, info) => + if (key._1 == target && info.isInstanceOf[ViewInfo]) { + ids.add(Identifier.of(key._1.toArray, key._2)) + } + } + ids.toArray(new Array[Identifier](0)) + } + + override def createView(ident: Identifier, info: ViewInfo): ViewInfo = { + val key = (ident.namespace().toSeq, ident.name()) + if (store.putIfAbsent(key, info) != null) { + throw new ViewAlreadyExistsException(ident) + } + info + } + + override def replaceView(ident: Identifier, info: ViewInfo): ViewInfo = { + val key = (ident.namespace().toSeq, ident.name()) + val existing = store.get(key) + if (existing == null || !existing.isInstanceOf[ViewInfo]) { + throw new NoSuchViewException(ident) + } + store.put(key, info) + info + } + + override def dropView(ident: Identifier): Boolean = { + val key = (ident.namespace().toSeq, ident.name()) + val existing = store.get(key) + if (existing == null || !existing.isInstanceOf[ViewInfo]) return false + store.remove(key) != null + } + + override def renameView(oldIdent: Identifier, newIdent: Identifier): Unit = { + val oldKey = (oldIdent.namespace().toSeq, oldIdent.name()) + val newKey = (newIdent.namespace().toSeq, newIdent.name()) + val existing = store.get(oldKey) + if (existing == null || !existing.isInstanceOf[ViewInfo]) { + throw new NoSuchViewException(oldIdent) + } + if (store.putIfAbsent(newKey, existing) != null) { + throw new ViewAlreadyExistsException(newIdent) + } + store.remove(oldKey) + } + + // Test-only accessors -------------------------------------------------------------- + + /** Returns the stored entry (table or view) for the identifier, or throws if missing. */ + def getStoredInfo(namespace: Array[String], name: String): TableInfo = { + Option(store.get((namespace.toSeq, name))).getOrElse { + throw new NoSuchTableException(Identifier.of(namespace, name)) + } + } + + /** Returns the stored ViewInfo, or throws if the entry is missing or is not a view. */ + def getStoredView(namespace: Array[String], name: String): ViewInfo = { + getStoredInfo(namespace, name) match { + case v: ViewInfo => v + case _ => throw new IllegalStateException( + s"stored entry at ${namespace.mkString(".")}.$name is not a view") + } + } + + // CatalogPlugin -------------------------------------------------------------------- + + private var catalogName: String = "" + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + catalogName = name + } + override def name(): String = catalogName +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index f8a9ce1d40d9..0dbcc3f51756 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -208,11 +208,21 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) output) => DescribeTableCommand(resolvedChild, ident, spec, isExtended, output) - case DescribeColumn( - ResolvedViewIdentifier(ident), column: UnresolvedAttribute, isExtended, output) => - // For views, the column will not be resolved by `ResolveReferences` because - // `ResolvedView` stores only the identifier. - DescribeColumnCommand(ident, column.nameParts, isExtended, output) + case DescribeColumn(ResolvedViewIdentifier(ident), column, isExtended, output) => + // `ResolvedPersistentView` exposes the view's schema as its `output`, so `ResolveReferences` + // typically resolves the column to an `Attribute` here. We also accept the legacy + // `UnresolvedAttribute` form (e.g. the parser referenced a non-existent column whose + // resolution was skipped) so the rewrite stays robust across analyzer ordering changes. + val nameParts = column match { + case u: UnresolvedAttribute => u.nameParts + case a: Attribute => a.qualifier :+ a.name + case Alias(child, _) => + throw QueryCompilationErrors.commandNotSupportNestedColumnError( + "DESC TABLE COLUMN", toPrettySQL(child)) + case _ => + throw SparkException.internalError(s"[BUG] unexpected column expression: $column") + } + DescribeColumnCommand(ident, nameParts, isExtended, output) case DescribeColumn(ResolvedV1TableIdentifier(ident), column, isExtended, output) => column match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index e02059d89692..6a00af8be9b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -27,7 +27,7 @@ import org.apache.spark.internal.LogKeys.EXPR import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedPersistentView, ResolvedTable, ResolvedTempView, UnresolvedAttribute} import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning, Expression, NamedExpression, Not, Or, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, DynamicPruning, Expression, NamedExpression, Not, Or, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ @@ -340,7 +340,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat AlterV2ViewSchemaBindingExec( catalog.asInstanceOf[ViewCatalog], ident, rpv.info, schemaMode) :: Nil - case RenameTable(ResolvedPersistentView(catalog, ident, _), newName, _) => + case RenameTable(ResolvedPersistentView(catalog, ident, _), newName, isView) => + // Reject `ALTER TABLE RENAME TO ...` -- the syntax says TABLE, but the resolved + // child is a view. Matches the v1 runtime check in `DDLUtils.verifyAlterTableType`. + if (!isView) { + throw QueryCompilationErrors.cannotAlterViewWithAlterTableError(ident.name()) + } RenameV2ViewExec( catalog.asInstanceOf[ViewCatalog], ident, newName.asIdentifier) :: Nil @@ -359,9 +364,21 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DescribeRelation(rpv @ ResolvedPersistentView(catalog, ident, _), isExtended, output) => DescribeV2ViewExec(output, catalog.name(), ident, rpv.info, isExtended) :: Nil - case DescribeColumn(rpv @ ResolvedPersistentView(_, _, _), column: UnresolvedAttribute, - isExtended, output) => - DescribeV2ViewColumnExec(output, rpv.info, column, isExtended) :: Nil + case DescribeColumn(rpv @ ResolvedPersistentView(_, _, _), column, isExtended, output) => + // `ResolvedPersistentView.output` exposes the view's schema, so `ResolveReferences` + // resolves the column against it -- meaning we typically receive an `Attribute` here. + // Accept the legacy `UnresolvedAttribute` form too, mirroring the v1 rewrite for + // session-catalog views in `ResolveSessionCatalog`. + val nameParts = column match { + case u: UnresolvedAttribute => u.nameParts + case a: Attribute => a.qualifier :+ a.name + case Alias(child, _) => + throw QueryCompilationErrors.commandNotSupportNestedColumnError( + "DESC TABLE COLUMN", toPrettySQL(child)) + case _ => + throw SparkException.internalError(s"[BUG] unexpected column expression: $column") + } + DescribeV2ViewColumnExec(output, rpv.info, nameParts, isExtended) :: Nil // Plans that resolve through `UnresolvedTableOrView` reach here with a // `ResolvedPersistentView` child for non-session v2 views (the v1 rewrite in diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala index 41831fbe2cce..f96317ff7e12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala @@ -21,7 +21,6 @@ import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIfNeeded, ResolveDefaultColumns} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumnsUtils.CURRENT_DEFAULT_COLUMN_METADATA_KEY @@ -204,21 +203,21 @@ case class DescribeV2ViewExec( } /** - * Physical plan node for DESCRIBE TABLE ... COLUMN on a v2 view. Mirrors the column-only - * branch of v1 `DescribeColumnCommand`: emit `col_name`, `data_type`, `comment` for the - * resolved field. v2 views do not carry column statistics, so the EXTENDED branch in v1 emits - * `NULL` for every stat row -- we follow the same shape. + * Physical plan node for DESCRIBE TABLE ... COLUMN on a v2 view. The column nameParts are + * extracted at strategy time from the (already-resolved) column expression on + * `DescribeColumn`, so this exec doesn't have to deal with resolution. v2 views don't carry + * column statistics, so the EXTENDED branch in v1 emits `NULL` for every stat row -- we + * follow the same shape. */ case class DescribeV2ViewColumnExec( output: Seq[Attribute], viewInfo: ViewInfo, - column: UnresolvedAttribute, + colNameParts: Seq[String], isExtended: Boolean) extends LeafV2CommandExec with SQLConfHelper { override protected def run(): Seq[InternalRow] = { val resolver = conf.resolver - val colNameParts = column.nameParts - val colName = column.name + val colName = colNameParts.mkString(".") if (colNameParts.length > 1) { throw QueryCompilationErrors.commandNotSupportNestedColumnError( "DESC TABLE COLUMN", colName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala index 38894b75ad92..79d003c54a69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala @@ -122,375 +122,11 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { assert(ct.viewCatalogAndNamespace.isEmpty) } - // --- CREATE VIEW on a plain TableCatalog -------------------------------- + // CREATE VIEW behavior tests live in the per-catalog triplet + // `sql.execution.command.{,v1/,v2/}.CreateViewSuite{,Base}`. - test("CREATE VIEW on a v2 catalog") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.my_view AS " + - "SELECT x FROM spark_catalog.default.t WHERE x > 1") - checkAnswer(spark.table("view_catalog.default.my_view"), Seq(Row(2), Row(3))) - } - } - - test("CREATE VIEW IF NOT EXISTS is a no-op when the view exists") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_ifne AS " + - "SELECT x FROM spark_catalog.default.t") - // Re-running with IF NOT EXISTS should not fail and should not change the view. - sql("CREATE VIEW IF NOT EXISTS view_catalog.default.v_ifne AS " + - "SELECT x + 100 AS x FROM spark_catalog.default.t") - checkAnswer(spark.table("view_catalog.default.v_ifne"), - Seq(Row(1), Row(2), Row(3))) - } - } - - test("CREATE VIEW without IF NOT EXISTS fails when the view exists") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_dup AS " + - "SELECT x FROM spark_catalog.default.t") - intercept[AnalysisException] { - sql("CREATE VIEW view_catalog.default.v_dup AS " + - "SELECT x FROM spark_catalog.default.t") - } - } - } - - test("CREATE OR REPLACE VIEW replaces an existing view") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_replace AS " + - "SELECT x FROM spark_catalog.default.t WHERE x > 10") - checkAnswer(spark.table("view_catalog.default.v_replace"), Seq.empty[Row]) - sql("CREATE OR REPLACE VIEW view_catalog.default.v_replace AS " + - "SELECT x FROM spark_catalog.default.t WHERE x > 1") - checkAnswer(spark.table("view_catalog.default.v_replace"), Seq(Row(2), Row(3))) - } - } - - test("CREATE VIEW on a catalog without ViewCatalog fails") { - withSQLConf( - "spark.sql.catalog.no_view_catalog" -> classOf[TestingTableOnlyCatalog].getName) { - val ex = intercept[AnalysisException] { - sql("CREATE VIEW no_view_catalog.default.v AS SELECT 1") - } - assert(ex.getCondition == "MISSING_CATALOG_ABILITY.VIEWS") - } - } - - test("CREATE VIEW rejects too-few / too-many user-specified columns") { - withTable("spark_catalog.default.t") { - Seq(1 -> 10).toDF("x", "y").write.saveAsTable("spark_catalog.default.t") - intercept[AnalysisException] { - sql("CREATE VIEW view_catalog.default.v_few (a) AS " + - "SELECT x, y FROM spark_catalog.default.t") - } - intercept[AnalysisException] { - sql("CREATE VIEW view_catalog.default.v_many (a, b, c) AS " + - "SELECT x, y FROM spark_catalog.default.t") - } - } - } - - test("CREATE VIEW rejects reference to a temporary function") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - spark.udf.register("temp_udf", (i: Int) => i + 1) - val ex = intercept[AnalysisException] { - sql("CREATE VIEW view_catalog.default.v_tempfn AS " + - "SELECT temp_udf(x) FROM spark_catalog.default.t") - } - assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).contains("temporary")) - } - } - - test("CREATE VIEW rejects reference to a temporary view") { - withTempView("tv") { - spark.range(3).createOrReplaceTempView("tv") - val ex = intercept[AnalysisException] { - sql("CREATE VIEW view_catalog.default.v_tempview AS SELECT id FROM tv") - } - assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).contains("temporary")) - } - } - - test("CREATE VIEW rejects reference to a temporary variable") { - withSessionVariable("temp_var") { - sql("DECLARE VARIABLE temp_var INT DEFAULT 1") - val ex = intercept[AnalysisException] { - sql("CREATE VIEW view_catalog.default.v_tempvar AS SELECT temp_var AS x") - } - assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).contains("temporary")) - } - } - - test("CREATE VIEW propagates DEFAULT COLLATION to TableInfo") { - withTable("spark_catalog.default.t") { - Seq("a", "b").toDF("col").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_coll DEFAULT COLLATION UTF8_BINARY AS " + - "SELECT col FROM spark_catalog.default.t") - // TestingTableViewCatalog stores the TableInfo verbatim, so the collation property is - // observable via the catalog-stored builder output. - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - val info = catalog.getStoredView(Array("default"), "v_coll") - assert(info.properties().get(TableCatalog.PROP_COLLATION) == "UTF8_BINARY") - } - } - - test("CREATE OR REPLACE VIEW detects cyclic view references") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_cycle_a AS " + - "SELECT x FROM spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_cycle_b AS " + - "SELECT x FROM view_catalog.default.v_cycle_a") - val ex = intercept[AnalysisException] { - sql("CREATE OR REPLACE VIEW view_catalog.default.v_cycle_a AS " + - "SELECT x FROM view_catalog.default.v_cycle_b") - } - assert(ex.getCondition == "RECURSIVE_VIEW") - } - } - - test("CREATE VIEW over a non-view table entry is rejected (plain TableCatalog)") { - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - val tableIdent = Identifier.of(Array("default"), "v_existing_table") - val tableInfo = new TableInfo.Builder() - .withSchema(new StructType().add("col", "string")) - .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) - .build() - catalog.createTable(tableIdent, tableInfo) - try { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - - // CREATE OR REPLACE VIEW must not silently destroy a non-view table -- v1 parity. - val replaceEx = intercept[AnalysisException] { - sql("CREATE OR REPLACE VIEW view_catalog.default.v_existing_table AS " + - "SELECT x FROM spark_catalog.default.t") - } - assert(replaceEx.getCondition == "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE") - - // Plain CREATE VIEW over a table surfaces TABLE_OR_VIEW_ALREADY_EXISTS, matching v1. - val createEx = intercept[AnalysisException] { - sql("CREATE VIEW view_catalog.default.v_existing_table AS " + - "SELECT x FROM spark_catalog.default.t") - } - assert(createEx.getCondition == "TABLE_OR_VIEW_ALREADY_EXISTS") - - // CREATE VIEW IF NOT EXISTS is a no-op -- the table entry is untouched. - sql("CREATE VIEW IF NOT EXISTS view_catalog.default.v_existing_table AS " + - "SELECT x FROM spark_catalog.default.t") - val stored = catalog.getStoredInfo(Array("default"), "v_existing_table") - assert(!stored.isInstanceOf[ViewInfo]) - assert(stored.properties().get(TableCatalog.PROP_TABLE_TYPE) == - TableSummary.EXTERNAL_TABLE_TYPE) - } - } finally { - catalog.dropTable(tableIdent) - } - } - - // --- ALTER VIEW --------------------------------------------------------- - - test("ALTER VIEW ... AS updates the view body on a v2 catalog") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_alter AS " + - "SELECT x FROM spark_catalog.default.t WHERE x > 10") - checkAnswer(spark.table("view_catalog.default.v_alter"), Seq.empty[Row]) - - sql("ALTER VIEW view_catalog.default.v_alter AS " + - "SELECT x FROM spark_catalog.default.t WHERE x > 1") - checkAnswer(spark.table("view_catalog.default.v_alter"), Seq(Row(2), Row(3))) - } - } - - test("ALTER VIEW on a missing view fails at analysis") { - // UnresolvedView resolves through lookupTableOrView and the missing view surfaces as an - // AnalysisException before we ever reach the v2 exec. The exact error condition (e.g. - // TABLE_OR_VIEW_NOT_FOUND) varies across Spark versions; we just assert we fail cleanly. - intercept[AnalysisException] { - sql("ALTER VIEW view_catalog.default.does_not_exist AS SELECT 1 AS x") - } - } - - test("ALTER VIEW rejects reference to a temporary function") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_alter_tempfn AS " + - "SELECT x FROM spark_catalog.default.t") - spark.udf.register("temp_udf_alter", (i: Int) => i + 1) - val ex = intercept[AnalysisException] { - sql("ALTER VIEW view_catalog.default.v_alter_tempfn AS " + - "SELECT temp_udf_alter(x) FROM spark_catalog.default.t") - } - assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).contains("temporary")) - } - } - - test("ALTER VIEW rejects reference to a temporary view") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_alter_tempview AS " + - "SELECT x FROM spark_catalog.default.t") - withTempView("tv_alter") { - spark.range(3).createOrReplaceTempView("tv_alter") - val ex = intercept[AnalysisException] { - sql("ALTER VIEW view_catalog.default.v_alter_tempview AS SELECT id FROM tv_alter") - } - assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).contains("temporary")) - } - } - } - - test("ALTER VIEW rejects reference to a temporary variable") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_alter_tempvar AS " + - "SELECT x FROM spark_catalog.default.t") - withSessionVariable("temp_var_alter") { - sql("DECLARE VARIABLE temp_var_alter INT DEFAULT 1") - val ex = intercept[AnalysisException] { - sql("ALTER VIEW view_catalog.default.v_alter_tempvar AS SELECT temp_var_alter AS x") - } - assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).contains("temporary")) - } - } - } - - test("ALTER VIEW preserves user-set TBLPROPERTIES") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_preserve " + - "TBLPROPERTIES ('mykey'='myvalue') AS " + - "SELECT x FROM spark_catalog.default.t") - sql("ALTER VIEW view_catalog.default.v_preserve AS " + - "SELECT x + 1 AS x FROM spark_catalog.default.t") - - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - val info = catalog.getStoredView(Array("default"), "v_preserve") - assert(info.properties().get("mykey") == "myvalue") - } - } - - test("CREATE VIEW stamps PROP_OWNER on the stored TableInfo") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_owner_create AS " + - "SELECT x FROM spark_catalog.default.t") - - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - val info = catalog.getStoredView(Array("default"), "v_owner_create") - // v2 CREATE VIEW stamps the current user into PROP_OWNER, matching v2 CREATE TABLE - // (via CatalogV2Util.withDefaultOwnership) and v1 CREATE VIEW (via CatalogTable.owner's - // default). Without this, the ALTER VIEW preservation test above would have nothing to - // carry forward on a v2-created view. - val owner = info.properties().get(TableCatalog.PROP_OWNER) - assert(owner != null && owner.nonEmpty, s"expected a non-empty owner, got: $owner") - } - } - - test("ALTER VIEW preserves PROP_OWNER (v1-parity)") { - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - val viewIdent = Identifier.of(Array("default"), "v_owner") - // Pre-seed a view whose stored ViewInfo carries an explicit owner. - val initialInfo = new ViewInfo.Builder() - .withSchema(new StructType().add("x", "int")) - .withQueryText("SELECT 1 AS x") - .withOwner("alice") - .withCurrentCatalog("spark_catalog") - .withCurrentNamespace(Array("default")) - .build() - catalog.createView(viewIdent, initialInfo) - try { - withTable("spark_catalog.default.t") { - Seq(2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("ALTER VIEW view_catalog.default.v_owner AS " + - "SELECT x FROM spark_catalog.default.t") - // v1 ALTER VIEW AS carries `owner` forward via `viewMeta.copy(...)`. v2 must match: - // the stored TableInfo after the ALTER should still have the original owner. - val info = catalog.getStoredView(Array("default"), "v_owner") - assert(info.properties().get(TableCatalog.PROP_OWNER) == "alice") - } - } finally { - catalog.dropTable(viewIdent) - } - } - - test("ALTER VIEW preserves SCHEMA EVOLUTION binding mode") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_evo WITH SCHEMA EVOLUTION AS " + - "SELECT x FROM spark_catalog.default.t") - sql("ALTER VIEW view_catalog.default.v_evo AS " + - "SELECT x + 1 AS x FROM spark_catalog.default.t") - - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - assert(catalog.getStoredView(Array("default"), "v_evo").schemaMode() == "EVOLUTION") - } - } - - test("ALTER VIEW re-captures the current session's SQL configs") { - withTable("spark_catalog.default.t") { - Seq("a", "b").toDF("col").write.saveAsTable("spark_catalog.default.t") - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { - sql("CREATE VIEW view_catalog.default.v_configs AS " + - "SELECT col FROM spark_catalog.default.t") - } - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - assert(catalog.getStoredView(Array("default"), "v_configs") - .sqlConfigs().get(SQLConf.ANSI_ENABLED.key) == "true") - - // ALTER under a different ANSI setting should replace the stored config, not merge. - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { - sql("ALTER VIEW view_catalog.default.v_configs AS " + - "SELECT col FROM spark_catalog.default.t WHERE col = 'b'") - } - assert(catalog.getStoredView(Array("default"), "v_configs") - .sqlConfigs().get(SQLConf.ANSI_ENABLED.key) == "false") - } - } - - test("CREATE OR REPLACE VIEW whose new body references a nonexistent table fails at " + - "analysis") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_replace_missing AS " + - "SELECT x FROM spark_catalog.default.t") - val ex = intercept[AnalysisException] { - sql("CREATE OR REPLACE VIEW view_catalog.default.v_replace_missing AS " + - "SELECT * FROM spark_catalog.default.does_not_exist") - } - assert(ex.getCondition == "TABLE_OR_VIEW_NOT_FOUND") - } - } - - test("ALTER VIEW on a catalog without ViewCatalog fails with MISSING_CATALOG_ABILITY") { - // ALTER VIEW's identifier is resolved via `UnresolvedView`, whose `viewOnly=true` path - // in `Analyzer.lookupTableOrView` rejects non-ViewCatalog catalogs up front with the - // expected error class -- before `loadTable` is even called. `TestingTableOnlyCatalog` - // happens to round-trip `default.v` as a view-typed MetadataTable, but that fixture - // is not actually consulted on this path. CREATE VIEW's capability check lives in - // `CheckViewReferences`; ALTER VIEW's lives in the analyzer gate. Both yield - // `MISSING_CATALOG_ABILITY.VIEWS`. - withSQLConf( - "spark.sql.catalog.no_view_catalog" -> classOf[TestingTableOnlyCatalog].getName) { - val ex = intercept[AnalysisException] { - sql("ALTER VIEW no_view_catalog.default.v AS SELECT 1 AS x") - } - assert(ex.getCondition == "MISSING_CATALOG_ABILITY.VIEWS") - } - } + // ALTER VIEW behavior tests live in the per-catalog triplet + // `sql.execution.command.{,v1/,v2/}.AlterViewAsSuite{,Base}`. // --- Pure ViewCatalog (no TableCatalog mixin) --------------------------- @@ -663,70 +299,9 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { assert(ex.getCondition == "UNSUPPORTED_FEATURE.TABLE_OPERATION", s"got ${ex.getCondition}") } - test("ALTER VIEW ... SET TBLPROPERTIES on a v2 view writes the properties") { - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - seedV2View("v_set_props") - sql("ALTER VIEW view_catalog.default.v_set_props SET TBLPROPERTIES ('k' = 'v')") - val stored = catalog.getStoredView(Array("default"), "v_set_props") - assert(stored.properties.get("k") == "v") - } - - test("ALTER VIEW ... UNSET TBLPROPERTIES on a v2 view drops the properties") { - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - seedV2View("v_unset_props") - sql("ALTER VIEW view_catalog.default.v_unset_props SET TBLPROPERTIES ('k' = 'v')") - sql("ALTER VIEW view_catalog.default.v_unset_props UNSET TBLPROPERTIES ('k')") - val stored = catalog.getStoredView(Array("default"), "v_unset_props") - assert(!stored.properties.containsKey("k")) - } - - test("ALTER VIEW ... WITH SCHEMA on a v2 view updates schemaMode") { - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - seedV2View("v_schema_binding") - sql("ALTER VIEW view_catalog.default.v_schema_binding WITH SCHEMA EVOLUTION") - val stored = catalog.getStoredView(Array("default"), "v_schema_binding") - assert(stored.schemaMode == "EVOLUTION") - } - - test("ALTER VIEW ... RENAME TO on a v2 view moves the entry") { - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - seedV2View("v_rename") - sql("ALTER VIEW view_catalog.default.v_rename RENAME TO view_catalog.default.v_renamed") - assert(!catalog.viewExists(Identifier.of(Array("default"), "v_rename"))) - assert(catalog.viewExists(Identifier.of(Array("default"), "v_renamed"))) - } - - test("SHOW CREATE TABLE on a v2 view emits CREATE VIEW") { - seedV2View("v_show_create") - val ddl = sql("SHOW CREATE TABLE view_catalog.default.v_show_create") - .collect().head.getString(0) - assert(ddl.startsWith("CREATE VIEW ")) - assert(ddl.contains("AS SELECT 1 AS x")) - } - - test("SHOW TBLPROPERTIES on a v2 view returns user properties") { - seedV2View("v_show_props") - sql("ALTER VIEW view_catalog.default.v_show_props SET TBLPROPERTIES ('k' = 'v')") - val rows = sql("SHOW TBLPROPERTIES view_catalog.default.v_show_props").collect() - assert(rows.exists(r => r.getString(0) == "k" && r.getString(1) == "v")) - } - - test("SHOW COLUMNS on a v2 view returns the column list") { - seedV2View("v_show_cols") - val cols = sql("SHOW COLUMNS IN view_catalog.default.v_show_cols") - .collect().map(_.getString(0)).toSeq - assert(cols == Seq("x")) - } - - test("DESCRIBE TABLE on a v2 view returns the schema") { - seedV2View("v_describe") - val rows = sql("DESCRIBE TABLE view_catalog.default.v_describe").collect() - assert(rows.exists(r => r.getString(0) == "x" && r.getString(1) == "int")) - } + // SET / UNSET / SCHEMA / RENAME / SHOW CREATE / SHOW TBLPROPERTIES / SHOW COLUMNS / + // DESCRIBE TABLE on a v2 view live in the per-catalog test triplets under + // `sql.execution.command.{,v1/,v2/}`; see e.g. AlterViewSetTblPropertiesSuite{,Base}. // These plans reach `DataSourceV2Strategy` with a `ResolvedPersistentView` child on a // non-session v2 view (because `ResolvedV1TableOrViewIdentifier` now skips non-session views). @@ -750,62 +325,8 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { "ANALYZE TABLE view_catalog.default.v_analyze_cols COMPUTE STATISTICS FOR COLUMNS x") } - // --- DROP VIEW on a v2 catalog -------------------------------- - - test("DROP VIEW on a ViewCatalog drops the view") { - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_drop AS " + - "SELECT x FROM spark_catalog.default.t") - assert(catalog.viewExists(Identifier.of(Array("default"), "v_drop"))) - sql("DROP VIEW view_catalog.default.v_drop") - assert(!catalog.viewExists(Identifier.of(Array("default"), "v_drop"))) - } - } - - test("DROP VIEW IF EXISTS on a v2 catalog is a no-op when the view is missing") { - // Exercises the `ifExists=true` path -- DropViewExec should not throw when the view - // doesn't exist on a ViewCatalog. - sql("DROP VIEW IF EXISTS view_catalog.default.v_never_existed") - } - - test("DROP VIEW on a non-view table entry is rejected (v1-parity)") { - // v1 `DropTableCommand(isView = true)` rejects a non-view target via - // `wrongCommandForObjectTypeError`. The v2 path must also refuse -- otherwise - // `DROP VIEW view_catalog.default.` would silently destroy the table's entry. - val catalog = spark.sessionState.catalogManager.catalog("view_catalog") - .asInstanceOf[TestingTableViewCatalog] - val tableIdent = Identifier.of(Array("default"), "t_not_a_view") - catalog.createTable( - tableIdent, - new TableInfo.Builder() - .withSchema(new StructType().add("col", "string")) - .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) - .build()) - try { - val ex = intercept[AnalysisException] { - sql("DROP VIEW view_catalog.default.t_not_a_view") - } - assert(ex.getCondition == "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE") - // The table entry must still be there -- DROP VIEW did not destroy it. - assert(catalog.tableExists(tableIdent)) - } finally { - catalog.dropTable(tableIdent) - } - } - - test("DROP VIEW on a catalog without ViewCatalog is rejected") { - withSQLConf( - "spark.sql.catalog.no_view_catalog" -> classOf[TestingTableOnlyCatalog].getName) { - val ex = intercept[AnalysisException] { - sql("DROP VIEW no_view_catalog.default.v") - } - // Preserves the pre-PR error surface for non-ViewCatalog catalogs. - assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).contains("views")) - } - } + // DROP VIEW behavior tests live in the per-catalog triplet + // `sql.execution.command.{,v1/,v2/}.DropViewSuite{,Base}`. // --- SHOW TABLES / SHOW VIEWS on a v2 catalog -------------------------------- @@ -834,49 +355,8 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { rows.foreach(r => assert(!r.getBoolean(2), s"isTemporary must be false: $r")) } - test("SHOW VIEWS on a v2 catalog returns only views") { - seedV2View("v_in_show_views") - seedV2Table("t_not_in_show_views") - val rows = sql("SHOW VIEWS IN view_catalog.default").collect() - val names = rows.map(_.getString(1)).toSet - assert(names.contains("v_in_show_views"), s"view missing: $names") - assert(!names.contains("t_not_in_show_views"), - s"non-view leaked into SHOW VIEWS: $names") - rows.foreach(r => assert(!r.getBoolean(2), s"isTemporary must be false for v2: $r")) - } - - test("SHOW VIEWS with LIKE pattern filters on the view name") { - seedV2View("v_foo") - seedV2View("v_bar") - val rows = sql("SHOW VIEWS IN view_catalog.default LIKE 'v_foo'").collect() - val names = rows.map(_.getString(1)).toSet - assert(names == Set("v_foo"), s"expected only v_foo, got $names") - } - - test("SHOW VIEWS on a catalog without ViewCatalog is rejected") { - withSQLConf( - "spark.sql.catalog.no_view_catalog" -> classOf[TestingTableOnlyCatalog].getName) { - val ex = intercept[AnalysisException] { - sql("SHOW VIEWS IN no_view_catalog.default") - } - assert(ex.getCondition == "MISSING_CATALOG_ABILITY.VIEWS") - } - } - - test("ALTER VIEW detects cyclic view references") { - withTable("spark_catalog.default.t") { - Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_alter_cycle_a AS " + - "SELECT x FROM spark_catalog.default.t") - sql("CREATE VIEW view_catalog.default.v_alter_cycle_b AS " + - "SELECT x FROM view_catalog.default.v_alter_cycle_a") - val ex = intercept[AnalysisException] { - sql("ALTER VIEW view_catalog.default.v_alter_cycle_a AS " + - "SELECT x FROM view_catalog.default.v_alter_cycle_b") - } - assert(ex.getCondition == "RECURSIVE_VIEW") - } - } + // SHOW VIEWS behavior tests live in the per-catalog triplet + // `sql.execution.command.{,v1/,v2/}.ShowViewsSuite{,Base}`. } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewAsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewAsSuiteBase.scala new file mode 100644 index 000000000000..1250ce84af1d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewAsSuiteBase.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import java.util.Locale + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} + +/** + * Unified tests for `ALTER VIEW ... AS` against V1 (session) and V2 view catalogs. + */ +trait AlterViewAsSuiteBase extends QueryTest with DDLCommandTestUtils { + import testImplicits._ + override val command: String = "ALTER VIEW ... AS" + + protected def namespace: String = "default" + + protected def withSourceTable(values: Int*)(body: => Unit): Unit = { + withTable("spark_catalog.default.alter_src") { + values.toSeq.toDF("x").write.saveAsTable("spark_catalog.default.alter_src") + body + } + } + + test("ALTER VIEW updates the body of an existing view") { + val view = s"$catalog.$namespace.v_alter_body" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $view AS " + + s"SELECT x FROM spark_catalog.default.alter_src WHERE x > 10") + checkAnswer(spark.table(view), Seq.empty[Row]) + sql(s"ALTER VIEW $view AS " + + s"SELECT x FROM spark_catalog.default.alter_src WHERE x > 1") + checkAnswer(spark.table(view), Seq(Row(2), Row(3))) + } + } + + test("ALTER VIEW on a missing view fails at analysis") { + val view = s"$catalog.$namespace.v_alter_missing" + intercept[AnalysisException] { + sql(s"ALTER VIEW $view AS SELECT 1 AS x") + } + } + + test("ALTER VIEW rejects reference to a temporary function") { + val view = s"$catalog.$namespace.v_alter_tempfn" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.alter_src") + spark.udf.register("temp_udf_alter", (i: Int) => i + 1) + val ex = intercept[AnalysisException] { + sql(s"ALTER VIEW $view AS " + + s"SELECT temp_udf_alter(x) FROM spark_catalog.default.alter_src") + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("temporary")) + } + } + + test("ALTER VIEW rejects reference to a temporary view") { + val view = s"$catalog.$namespace.v_alter_tempview" + withSourceTable(1) { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.alter_src") + withTempView("tv_alter") { + spark.range(3).createOrReplaceTempView("tv_alter") + val ex = intercept[AnalysisException] { + sql(s"ALTER VIEW $view AS SELECT id AS x FROM tv_alter") + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("temporary")) + } + } + } + + test("ALTER VIEW rejects reference to a temporary variable") { + val view = s"$catalog.$namespace.v_alter_tempvar" + withSourceTable(1) { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.alter_src") + sql("DECLARE OR REPLACE VARIABLE temp_var_alter INT DEFAULT 1") + try { + val ex = intercept[AnalysisException] { + sql(s"ALTER VIEW $view AS SELECT temp_var_alter AS x") + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("temporary")) + } finally { + sql("DROP TEMPORARY VARIABLE IF EXISTS temp_var_alter") + } + } + } + + test("ALTER VIEW preserves user-set TBLPROPERTIES") { + val view = s"$catalog.$namespace.v_alter_keep_props" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $view " + + s"TBLPROPERTIES ('key1' = 'val1') AS " + + s"SELECT x FROM spark_catalog.default.alter_src") + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('key2' = 'val2')") + sql(s"ALTER VIEW $view AS SELECT x + 1 AS x FROM spark_catalog.default.alter_src") + val rows = sql(s"SHOW TBLPROPERTIES $view").collect() + val pairs = rows.map(r => r.getString(0) -> r.getString(1)).toMap + assert(pairs.get("key1").contains("val1")) + assert(pairs.get("key2").contains("val2")) + } + } + + test("ALTER VIEW preserves SCHEMA EVOLUTION binding mode") { + val view = s"$catalog.$namespace.v_alter_keep_schema_mode" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $view WITH SCHEMA EVOLUTION AS " + + s"SELECT x FROM spark_catalog.default.alter_src") + sql(s"ALTER VIEW $view AS SELECT x + 1 AS x FROM spark_catalog.default.alter_src") + val ddl = sql(s"SHOW CREATE TABLE $view").collect().head.getString(0) + assert(ddl.contains("WITH SCHEMA EVOLUTION"), + s"schema-binding mode lost across ALTER VIEW AS:\n$ddl") + } + } + + test("CREATE OR REPLACE VIEW with a body referencing a missing table fails") { + val view = s"$catalog.$namespace.v_alter_bad_body" + withSourceTable(1) { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.alter_src") + intercept[AnalysisException] { + sql(s"CREATE OR REPLACE VIEW $view AS SELECT x FROM does_not_exist_at_all") + } + } + } + + test("ALTER VIEW detects a direct cyclic reference") { + val a = s"$catalog.$namespace.v_alter_cycle_a" + val b = s"$catalog.$namespace.v_alter_cycle_b" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $a AS SELECT x FROM spark_catalog.default.alter_src") + sql(s"CREATE VIEW $b AS SELECT x FROM $a") + val ex = intercept[AnalysisException] { + sql(s"ALTER VIEW $a AS SELECT x FROM $b") + } + assert(ex.getCondition == "RECURSIVE_VIEW") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala new file mode 100644 index 000000000000..b9f694c92fe3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} + +/** + * Unified tests for `ALTER VIEW ... RENAME TO` against V1 (session) and V2 view catalogs. + */ +trait AlterViewRenameSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "ALTER VIEW ... RENAME TO" + + protected def namespace: String = "default" + + protected def createView(view: String): Unit = { + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + } + + protected def viewExists(qualified: String): Boolean = { + val parts = qualified.split('.').toSeq + val nsAndView = parts.tail + val ns = nsAndView.init.mkString(".") + val name = nsAndView.last + sql(s"SHOW VIEWS IN $catalog.$ns").collect().exists(_.getString(1) == name) + } + + test("rename moves the entry") { + val src = s"$catalog.$namespace.v_rename_src" + val dstName = "v_rename_dst" + createView(src) + // v1 `AlterTableRenameCommand` requires a 1- or 2-part target identifier; a single name + // renames in the same namespace. v2 accepts either form. Use the unqualified form so the + // base test runs against both paths. + sql(s"ALTER VIEW $src RENAME TO $dstName") + assert(!viewExists(src), s"$src should be gone after rename") + assert(viewExists(s"$catalog.$namespace.$dstName"), + s"$catalog.$namespace.$dstName should exist after rename") + } + + test("rename preserves the view body") { + val src = s"$catalog.$namespace.v_rename_body_src" + val dstName = "v_rename_body_dst" + sql(s"CREATE VIEW $src AS SELECT 7 AS answer") + sql(s"ALTER VIEW $src RENAME TO $dstName") + checkAnswer(sql(s"SELECT * FROM $catalog.$namespace.$dstName"), Row(7)) + } + + test("renaming to an existing name fails") { + val src = s"$catalog.$namespace.v_rename_collide_src" + val dst = s"$catalog.$namespace.v_rename_collide_dst" + createView(src) + createView(dst) + intercept[AnalysisException] { + sql(s"ALTER VIEW $src RENAME TO v_rename_collide_dst") + } + } + + test("ALTER TABLE syntax on a view is rejected (use ALTER VIEW)") { + // `ALTER TABLE x RENAME TO y` and `ALTER VIEW x RENAME TO y` use the same parser entry + // (`UnresolvedTableOrView` + `isView` flag); when the resolved child is a view but the + // syntax says TABLE, error with EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW. v1 enforces this in + // `DDLUtils.verifyAlterTableType`; v2 enforces it in DataSourceV2Strategy. + val view = s"$catalog.$namespace.v_rename_wrong_syntax" + createView(view) + val ex = intercept[AnalysisException] { + sql(s"ALTER TABLE $view RENAME TO v_rename_wrong_syntax_dst") + } + assert(ex.getCondition.startsWith("EXPECT_TABLE_NOT_VIEW"), + s"unexpected error condition: ${ex.getCondition}") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSchemaBindingSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSchemaBindingSuiteBase.scala new file mode 100644 index 000000000000..25bbe73e5b76 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSchemaBindingSuiteBase.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `ALTER VIEW ... WITH SCHEMA` against V1 (session) and V2 view catalogs. + */ +trait AlterViewSchemaBindingSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "ALTER VIEW ... WITH SCHEMA" + + protected def namespace: String = "default" + + protected def createView(view: String): Unit = { + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + } + + protected def schemaModeOf(view: String): String = { + val rows = sql(s"SHOW CREATE TABLE $view").collect() + val ddl = rows.head.getString(0) + // Extract the WITH SCHEMA clause if present. + val pattern = """WITH SCHEMA\s+(BINDING|COMPENSATION|EVOLUTION|TYPE EVOLUTION)""".r + pattern.findFirstMatchIn(ddl).map(_.group(1)).getOrElse("BINDING") + } + + test("set EVOLUTION") { + val view = s"$catalog.$namespace.v_schema_evolve" + createView(view) + sql(s"ALTER VIEW $view WITH SCHEMA EVOLUTION") + assert(schemaModeOf(view) == "EVOLUTION") + } + + test("set COMPENSATION") { + val view = s"$catalog.$namespace.v_schema_compensate" + createView(view) + sql(s"ALTER VIEW $view WITH SCHEMA COMPENSATION") + assert(schemaModeOf(view) == "COMPENSATION") + } + + test("set BINDING (default)") { + val view = s"$catalog.$namespace.v_schema_binding" + createView(view) + sql(s"ALTER VIEW $view WITH SCHEMA EVOLUTION") + sql(s"ALTER VIEW $view WITH SCHEMA BINDING") + assert(schemaModeOf(view) == "BINDING") + } + + test("WITH SCHEMA does not change the view body") { + val view = s"$catalog.$namespace.v_schema_body_intact" + sql(s"CREATE VIEW $view AS SELECT 7 AS x") + sql(s"ALTER VIEW $view WITH SCHEMA EVOLUTION") + checkAnswer(sql(s"SELECT * FROM $view"), org.apache.spark.sql.Row(7)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala new file mode 100644 index 000000000000..68119d7d8e73 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.{AnalysisException, QueryTest} + +/** + * This base suite contains unified tests for the `ALTER VIEW ... SET TBLPROPERTIES` command + * that check V1 (session) and V2 view catalogs: + * + * - V2 view catalog: `org.apache.spark.sql.execution.command.v2.AlterViewSetTblPropertiesSuite` + * - V1 (session) view catalog: + * `org.apache.spark.sql.execution.command.v1.AlterViewSetTblPropertiesSuite` + */ +trait AlterViewSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "ALTER VIEW ... SET TBLPROPERTIES" + + protected def namespace: String = "default" + + protected def createView(view: String): Unit = { + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + } + + /** Reads back the property value visible via SHOW TBLPROPERTIES, or `None` if absent. */ + protected def lookupProperty(view: String, key: String): Option[String] = { + // SHOW TBLPROPERTIES ('key') always returns a single row; on a missing key the row + // carries a "does not have property" placeholder. Iterate the full property listing + // instead so absence is unambiguous. + sql(s"SHOW TBLPROPERTIES $view").collect() + .find(_.getString(0) == key) + .map(_.getString(1)) + } + + test("set a single property") { + val view = s"$catalog.$namespace.v_set_one" + createView(view) + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v')") + assert(lookupProperty(view, "k").contains("v")) + } + + test("set multiple properties at once") { + val view = s"$catalog.$namespace.v_set_many" + createView(view) + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('a' = '1', 'b' = '2')") + assert(lookupProperty(view, "a").contains("1")) + assert(lookupProperty(view, "b").contains("2")) + } + + test("setting overwrites existing property") { + val view = s"$catalog.$namespace.v_set_overwrite" + createView(view) + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v1')") + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v2')") + assert(lookupProperty(view, "k").contains("v2")) + } + + test("missing view raises a clean analysis error") { + val view = s"$catalog.$namespace.v_missing_set" + val ex = intercept[AnalysisException] { + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v')") + } + // Both v1 and v2 paths surface a TABLE_OR_VIEW_NOT_FOUND-shaped error here; the exact + // condition string can differ slightly between paths, so just assert non-empty message. + assert(ex.getMessage.contains(view.split('.').last)) + } + + test("show TBLPROPERTIES reflects the set property") { + val view = s"$catalog.$namespace.v_show_after_set" + createView(view) + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v')") + val rows = sql(s"SHOW TBLPROPERTIES $view").collect() + assert(rows.exists(r => r.getString(0) == "k" && r.getString(1) == "v"), + s"property k=v missing from SHOW TBLPROPERTIES: ${rows.mkString(", ")}") + } + + test("read-after-write returns user-set value") { + val view = s"$catalog.$namespace.v_read_after_set" + createView(view) + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('answer' = '42')") + assert(lookupProperty(view, "answer").contains("42")) + val all = sql(s"SHOW TBLPROPERTIES $view").collect() + assert(all.length >= 1, s"expected at least one property row, got: ${all.mkString(", ")}") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewUnsetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewUnsetTblPropertiesSuiteBase.scala new file mode 100644 index 000000000000..806e2aaa254f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewUnsetTblPropertiesSuiteBase.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `ALTER VIEW ... UNSET TBLPROPERTIES` against V1 (session) and V2 view + * catalogs. + */ +trait AlterViewUnsetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "ALTER VIEW ... UNSET TBLPROPERTIES" + + protected def namespace: String = "default" + + protected def createViewWithProps(view: String, propPairs: (String, String)*): Unit = { + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + if (propPairs.nonEmpty) { + val props = propPairs.map { case (k, v) => s"'$k' = '$v'" }.mkString(", ") + sql(s"ALTER VIEW $view SET TBLPROPERTIES ($props)") + } + } + + protected def lookupProperty(view: String, key: String): Option[String] = { + sql(s"SHOW TBLPROPERTIES $view").collect() + .find(_.getString(0) == key) + .map(_.getString(1)) + } + + test("unset a single property") { + val view = s"$catalog.$namespace.v_unset_one" + createViewWithProps(view, "k" -> "v") + sql(s"ALTER VIEW $view UNSET TBLPROPERTIES ('k')") + assert(lookupProperty(view, "k").isEmpty) + } + + test("unset preserves other properties") { + val view = s"$catalog.$namespace.v_unset_keeps_others" + createViewWithProps(view, "k" -> "v", "other" -> "stay") + sql(s"ALTER VIEW $view UNSET TBLPROPERTIES ('k')") + assert(lookupProperty(view, "k").isEmpty) + assert(lookupProperty(view, "other").contains("stay")) + } + + test("unset multiple keys at once") { + val view = s"$catalog.$namespace.v_unset_many" + createViewWithProps(view, "a" -> "1", "b" -> "2", "c" -> "3") + sql(s"ALTER VIEW $view UNSET TBLPROPERTIES ('a', 'b')") + assert(lookupProperty(view, "a").isEmpty) + assert(lookupProperty(view, "b").isEmpty) + assert(lookupProperty(view, "c").contains("3")) + } + + test("unset with IF EXISTS on a missing key is a no-op") { + val view = s"$catalog.$namespace.v_unset_if_exists" + createViewWithProps(view, "k" -> "v") + sql(s"ALTER VIEW $view UNSET TBLPROPERTIES IF EXISTS ('not_there')") + // Existing property remains. + assert(lookupProperty(view, "k").contains("v")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala new file mode 100644 index 000000000000..b5c2e584402c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import java.util.Locale + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} + +/** + * Unified tests for `CREATE VIEW` against V1 (session) and V2 view catalogs. Source data lives + * at `spark_catalog.default.t` for both paths. + */ +trait CreateViewSuiteBase extends QueryTest with DDLCommandTestUtils { + import testImplicits._ + override val command: String = "CREATE VIEW" + + protected def namespace: String = "default" + + protected def withSourceTable(values: Int*)(body: => Unit): Unit = { + withTable("spark_catalog.default.src") { + values.toSeq.toDF("x").write.saveAsTable("spark_catalog.default.src") + body + } + } + + test("CREATE VIEW persists the body and the SELECT round-trips") { + val view = s"$catalog.$namespace.v_create_basic" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.src WHERE x > 1") + checkAnswer(spark.table(view), Seq(Row(2), Row(3))) + } + } + + test("CREATE VIEW IF NOT EXISTS is a no-op when the view already exists") { + val view = s"$catalog.$namespace.v_create_ifne" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.src") + // Re-running with IF NOT EXISTS must succeed without changing the body. + sql(s"CREATE VIEW IF NOT EXISTS $view AS " + + s"SELECT x + 100 AS x FROM spark_catalog.default.src") + checkAnswer(spark.table(view), Seq(Row(1), Row(2), Row(3))) + } + } + + test("CREATE VIEW without IF NOT EXISTS fails when the view exists") { + val view = s"$catalog.$namespace.v_create_dup" + withSourceTable(1) { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.src") + intercept[AnalysisException] { + sql(s"CREATE VIEW $view AS SELECT x FROM spark_catalog.default.src") + } + } + } + + test("CREATE OR REPLACE VIEW replaces the body of an existing view") { + val view = s"$catalog.$namespace.v_create_replace" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $view AS " + + s"SELECT x FROM spark_catalog.default.src WHERE x > 10") + checkAnswer(spark.table(view), Seq.empty[Row]) + sql(s"CREATE OR REPLACE VIEW $view AS " + + s"SELECT x FROM spark_catalog.default.src WHERE x > 1") + checkAnswer(spark.table(view), Seq(Row(2), Row(3))) + } + } + + test("CREATE VIEW with a user-specified column list aliases the output") { + val view = s"$catalog.$namespace.v_create_cols" + withSourceTable(1, 2) { + sql(s"CREATE VIEW $view (alpha, beta) AS " + + s"SELECT x AS xa, (x + 1) AS xb FROM spark_catalog.default.src") + val cols = spark.table(view).schema.fieldNames.toSeq + assert(cols == Seq("alpha", "beta")) + } + } + + test("CREATE VIEW rejects too-few user-specified columns") { + val view = s"$catalog.$namespace.v_create_few" + withSourceTable(1) { + intercept[AnalysisException] { + sql(s"CREATE VIEW $view (a) AS " + + s"SELECT x AS xa, x AS xb FROM spark_catalog.default.src") + } + } + } + + test("CREATE VIEW rejects too-many user-specified columns") { + val view = s"$catalog.$namespace.v_create_many" + withSourceTable(1) { + intercept[AnalysisException] { + sql(s"CREATE VIEW $view (a, b, c) AS SELECT x FROM spark_catalog.default.src") + } + } + } + + test("CREATE VIEW rejects reference to a temporary function") { + val view = s"$catalog.$namespace.v_create_tempfn" + withSourceTable(1, 2, 3) { + spark.udf.register("temp_udf_create", (i: Int) => i + 1) + val ex = intercept[AnalysisException] { + sql(s"CREATE VIEW $view AS " + + s"SELECT temp_udf_create(x) FROM spark_catalog.default.src") + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("temporary")) + } + } + + test("CREATE VIEW rejects reference to a temporary view") { + val view = s"$catalog.$namespace.v_create_tempview" + withTempView("tv_create") { + spark.range(3).createOrReplaceTempView("tv_create") + val ex = intercept[AnalysisException] { + sql(s"CREATE VIEW $view AS SELECT id FROM tv_create") + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("temporary")) + } + } + + test("CREATE VIEW rejects reference to a temporary variable") { + val view = s"$catalog.$namespace.v_create_tempvar" + sql("DECLARE OR REPLACE VARIABLE temp_var_create INT DEFAULT 1") + try { + val ex = intercept[AnalysisException] { + sql(s"CREATE VIEW $view AS SELECT temp_var_create AS x") + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("temporary")) + } finally { + sql("DROP TEMPORARY VARIABLE IF EXISTS temp_var_create") + } + } + + test("CREATE OR REPLACE VIEW detects a direct cyclic reference") { + val a = s"$catalog.$namespace.v_create_cycle_a" + val b = s"$catalog.$namespace.v_create_cycle_b" + withSourceTable(1, 2, 3) { + sql(s"CREATE VIEW $a AS SELECT x FROM spark_catalog.default.src") + sql(s"CREATE VIEW $b AS SELECT x FROM $a") + val ex = intercept[AnalysisException] { + sql(s"CREATE OR REPLACE VIEW $a AS SELECT x FROM $b") + } + assert(ex.getCondition == "RECURSIVE_VIEW") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewColumnSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewColumnSuiteBase.scala new file mode 100644 index 000000000000..5969354c443f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewColumnSuiteBase.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `DESCRIBE TABLE ... ` against a view, on V1 (session) and V2 view + * catalogs. + */ +trait DescribeViewColumnSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "DESCRIBE TABLE COLUMN on view" + + protected def namespace: String = "default" + + test("emits col_name / data_type / comment rows") { + val view = s"$catalog.$namespace.v_desc_col_basic" + sql(s"CREATE VIEW $view AS SELECT 1 AS a") + val rows = sql(s"DESCRIBE TABLE $view a").collect() + val labels = rows.map(_.getString(0)).toSet + assert(labels.contains("col_name")) + assert(labels.contains("data_type")) + assert(labels.contains("comment")) + } + + test("data_type matches the column type") { + val view = s"$catalog.$namespace.v_desc_col_type" + sql(s"CREATE VIEW $view AS SELECT 1 AS a, 'x' AS b") + val rows = sql(s"DESCRIBE TABLE $view b").collect() + val pairs = rows.map(r => r.getString(0) -> r.getString(1)).toMap + assert(pairs.get("col_name").contains("b")) + assert(pairs.get("data_type").contains("string")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala new file mode 100644 index 000000000000..ff745a9b8028 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `DESCRIBE TABLE` against a view, on V1 (session) and V2 view catalogs. + */ +trait DescribeViewSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "DESCRIBE TABLE on view" + + protected def namespace: String = "default" + + test("describe emits one row per column") { + val view = s"$catalog.$namespace.v_describe_basic" + sql(s"CREATE VIEW $view AS SELECT 1 AS a, 'x' AS b") + val rows = sql(s"DESCRIBE TABLE $view").collect() + val cols = rows.map(r => r.getString(0) -> r.getString(1)).toMap + assert(cols.get("a").contains("int")) + assert(cols.get("b").contains("string")) + } + + test("describe extended emits a detailed-info block for the view") { + val view = s"$catalog.$namespace.v_describe_extended" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + val rows = sql(s"DESCRIBE TABLE EXTENDED $view").collect().map(_.getString(0)) + // v1 and v2 paths render slightly different headers ('# Detailed Table Information' vs + // '# Detailed View Information'); accept either. + assert( + rows.contains("# Detailed Table Information") || + rows.contains("# Detailed View Information"), + s"expected a detailed-info block in:\n${rows.mkString("\n")}") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala new file mode 100644 index 000000000000..ffa84f32e555 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.{AnalysisException, QueryTest} + +/** + * Unified tests for `DROP VIEW` against V1 (session) and V2 view catalogs. + */ +trait DropViewSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "DROP VIEW" + + protected def namespace: String = "default" + + protected def viewExists(qualified: String): Boolean = { + val parts = qualified.split('.').toSeq + val nsAndView = parts.tail + val ns = nsAndView.init.mkString(".") + val name = nsAndView.last + sql(s"SHOW VIEWS IN $catalog.$ns").collect().exists(_.getString(1) == name) + } + + test("drop existing view") { + val view = s"$catalog.$namespace.v_drop_basic" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + assert(viewExists(view)) + sql(s"DROP VIEW $view") + assert(!viewExists(view)) + } + + test("drop missing view fails without IF EXISTS") { + val view = s"$catalog.$namespace.v_drop_missing" + intercept[AnalysisException] { + sql(s"DROP VIEW $view") + } + } + + test("drop with IF EXISTS is a no-op when missing") { + sql(s"DROP VIEW IF EXISTS $catalog.$namespace.v_drop_never_existed") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala new file mode 100644 index 000000000000..37cb627468cf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `SHOW CREATE TABLE` against a view, on V1 (session) and V2 view catalogs. + */ +trait ShowCreateViewSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "SHOW CREATE TABLE on view" + + protected def namespace: String = "default" + + test("emits CREATE VIEW prefix") { + val view = s"$catalog.$namespace.v_show_create_basic" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + val ddl = sql(s"SHOW CREATE TABLE $view").collect().head.getString(0) + assert(ddl.startsWith("CREATE VIEW "), s"unexpected DDL: $ddl") + assert(ddl.contains("AS SELECT 1 AS x"), s"unexpected DDL: $ddl") + } + + test("includes user-set TBLPROPERTIES") { + val view = s"$catalog.$namespace.v_show_create_props" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v')") + val ddl = sql(s"SHOW CREATE TABLE $view").collect().head.getString(0) + assert(ddl.contains("'k' = 'v'"), s"property missing in DDL: $ddl") + } + + test("renders the column list") { + val view = s"$catalog.$namespace.v_show_create_cols" + sql(s"CREATE VIEW $view (a, b) AS SELECT 1, 2") + val ddl = sql(s"SHOW CREATE TABLE $view").collect().head.getString(0) + assert(ddl.contains("a")) + assert(ddl.contains("b")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala new file mode 100644 index 000000000000..7273db1587d7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `SHOW COLUMNS` against a view, on V1 (session) and V2 view catalogs. + */ +trait ShowViewColumnsSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "SHOW COLUMNS on view" + + protected def namespace: String = "default" + + test("returns one row per column") { + val view = s"$catalog.$namespace.v_show_cols_basic" + sql(s"CREATE VIEW $view AS SELECT 1 AS a, 'x' AS b") + val cols = sql(s"SHOW COLUMNS IN $view").collect().map(_.getString(0)).toSeq + assert(cols == Seq("a", "b")) + } + + test("respects user-specified column list on the view") { + val view = s"$catalog.$namespace.v_show_cols_aliased" + sql(s"CREATE VIEW $view (alpha, beta) AS SELECT 1, 2") + val cols = sql(s"SHOW COLUMNS IN $view").collect().map(_.getString(0)).toSeq + assert(cols == Seq("alpha", "beta")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewPropertiesSuiteBase.scala new file mode 100644 index 000000000000..873b65a42ddf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewPropertiesSuiteBase.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `SHOW TBLPROPERTIES` against a view, on V1 (session) and V2 view catalogs. + */ +trait ShowViewPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "SHOW TBLPROPERTIES on view" + + protected def namespace: String = "default" + + test("returns user-set property by key") { + val view = s"$catalog.$namespace.v_show_props_one" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v')") + val rows = sql(s"SHOW TBLPROPERTIES $view ('k')").collect() + // SHOW TBLPROPERTIES ('key') returns either (value) or (key, value) -- check the + // last column either way. + assert(rows.head.getString(rows.head.length - 1) == "v") + } + + test("returns all user-set properties") { + val view = s"$catalog.$namespace.v_show_props_all" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('a' = '1', 'b' = '2')") + val rows = sql(s"SHOW TBLPROPERTIES $view").collect() + val pairs = rows.map(r => r.getString(0) -> r.getString(1)).toMap + assert(pairs.get("a").contains("1")) + assert(pairs.get("b").contains("2")) + } + + test("missing key returns the default not-found row") { + val view = s"$catalog.$namespace.v_show_props_missing_key" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + val rows = sql(s"SHOW TBLPROPERTIES $view ('not_there')").collect() + val value = rows.head.getString(rows.head.length - 1) + assert(value.contains("does not have property"), + s"expected a not-found message, got: $value") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala new file mode 100644 index 000000000000..384635857865 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.QueryTest + +/** + * Unified tests for `SHOW VIEWS` against V1 (session) and V2 view catalogs. + */ +trait ShowViewsSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command: String = "SHOW VIEWS" + + protected def namespace: String = "default" + + test("returns user-created views") { + sql(s"CREATE VIEW $catalog.$namespace.v_show_views_a AS SELECT 1 AS x") + sql(s"CREATE VIEW $catalog.$namespace.v_show_views_b AS SELECT 2 AS x") + val rows = sql(s"SHOW VIEWS IN $catalog.$namespace").collect() + val names = rows.map(_.getString(1)).toSet + assert(names.contains("v_show_views_a"), s"v_show_views_a missing: $names") + assert(names.contains("v_show_views_b"), s"v_show_views_b missing: $names") + } + + test("LIKE pattern filters by name") { + sql(s"CREATE VIEW $catalog.$namespace.show_views_match AS SELECT 1 AS x") + sql(s"CREATE VIEW $catalog.$namespace.show_views_skip AS SELECT 1 AS x") + val rows = sql(s"SHOW VIEWS IN $catalog.$namespace LIKE 'show_views_match'").collect() + val names = rows.map(_.getString(1)).toSet + assert(names.contains("show_views_match")) + assert(!names.contains("show_views_skip")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewAsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewAsSuite.scala new file mode 100644 index 000000000000..c2f6851b2ae0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewAsSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class AlterViewAsSuite extends command.AlterViewAsSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewRenameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewRenameSuite.scala new file mode 100644 index 000000000000..9c2bcb2b90f4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewRenameSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class AlterViewRenameSuite + extends command.AlterViewRenameSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSchemaBindingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSchemaBindingSuite.scala new file mode 100644 index 000000000000..39e6e708403a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSchemaBindingSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class AlterViewSchemaBindingSuite + extends command.AlterViewSchemaBindingSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSetTblPropertiesSuite.scala new file mode 100644 index 000000000000..477c7288d018 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewSetTblPropertiesSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +/** + * The class contains tests for the `ALTER VIEW ... SET TBLPROPERTIES` command on V1 + * (session-catalog) views. + */ +class AlterViewSetTblPropertiesSuite + extends command.AlterViewSetTblPropertiesSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewUnsetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewUnsetTblPropertiesSuite.scala new file mode 100644 index 000000000000..a1caab3de1c6 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterViewUnsetTblPropertiesSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class AlterViewUnsetTblPropertiesSuite + extends command.AlterViewUnsetTblPropertiesSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala new file mode 100644 index 000000000000..383fbdb2eca4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewColumnSuite.scala new file mode 100644 index 000000000000..ce72f5e839d8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewColumnSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class DescribeViewColumnSuite + extends command.DescribeViewColumnSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewSuite.scala new file mode 100644 index 000000000000..3d9ed0bd7b48 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeViewSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class DescribeViewSuite + extends command.DescribeViewSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala new file mode 100644 index 000000000000..c8805d67391b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateViewSuite.scala new file mode 100644 index 000000000000..17b5009403c1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateViewSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class ShowCreateViewSuite + extends command.ShowCreateViewSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewColumnsSuite.scala new file mode 100644 index 000000000000..960092f45238 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewColumnsSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class ShowViewColumnsSuite + extends command.ShowViewColumnsSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewPropertiesSuite.scala new file mode 100644 index 000000000000..e76dc5e6f7cc --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewPropertiesSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class ShowViewPropertiesSuite + extends command.ShowViewPropertiesSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewsSuite.scala new file mode 100644 index 000000000000..53d218cda3f4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowViewsSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.execution.command + +class ShowViewsSuite extends command.ShowViewsSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ViewCommandSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ViewCommandSuiteBase.scala new file mode 100644 index 000000000000..499705288079 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ViewCommandSuiteBase.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +/** + * Settings for v1 view command test suites. The session catalog hosts views natively; reuse + * v1 [[CommandSuiteBase]] so view tests inherit its `checkLocation` and other helpers. + */ +trait ViewCommandSuiteBase extends CommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala new file mode 100644 index 000000000000..08953bfa96d1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, TableCatalog} +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.internal.SQLConf + +class AlterViewAsSuite extends command.AlterViewAsSuiteBase with ViewCommandSuiteBase { + + test("V2: ALTER VIEW preserves PROP_OWNER (v1-parity)") { + val view = s"$catalog.$namespace.v2_alter_keep_owner" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + val ownerBefore = viewCatalog.getStoredView(Array(namespace), "v2_alter_keep_owner") + .properties().get(TableCatalog.PROP_OWNER) + sql(s"ALTER VIEW $view AS SELECT 2 AS x") + val ownerAfter = viewCatalog.getStoredView(Array(namespace), "v2_alter_keep_owner") + .properties().get(TableCatalog.PROP_OWNER) + assert(ownerBefore == ownerAfter) + } + + test("V2: ALTER VIEW re-captures the current session's SQL configs") { + val view = s"$catalog.$namespace.v2_alter_reconfig" + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + } + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + sql(s"ALTER VIEW $view AS SELECT 2 AS x") + } + val stored = viewCatalog.getStoredView(Array(namespace), "v2_alter_reconfig") + val captured = stored.sqlConfigs().get(SQLConf.ANSI_ENABLED.key) + assert(captured == "true", + s"expected ALTER VIEW to re-capture ansi=true; got $captured") + } + + test("V2: ALTER VIEW on non-ViewCatalog catalog fails with MISSING_CATALOG_ABILITY") { + withSQLConf( + "spark.sql.catalog.no_view_alter_cat" -> classOf[BasicInMemoryTableCatalog].getName) { + val ex = intercept[AnalysisException] { + sql("ALTER VIEW no_view_alter_cat.default.does_not_matter AS SELECT 1") + } + assert(ex.getCondition == "MISSING_CATALOG_ABILITY.VIEWS") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala new file mode 100644 index 000000000000..53edd698e394 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.execution.command + +class AlterViewRenameSuite + extends command.AlterViewRenameSuiteBase with ViewCommandSuiteBase { + + test("V2: catalog state moves the entry between identifiers") { + val src = s"$catalog.$namespace.v2_rename_src" + createView(src) + sql(s"ALTER VIEW $src RENAME TO v2_rename_dst") + assert(!viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_rename_src"))) + assert(viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_rename_dst"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala new file mode 100644 index 000000000000..14112a6e0c87 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +class AlterViewSchemaBindingSuite + extends command.AlterViewSchemaBindingSuiteBase with ViewCommandSuiteBase { + + test("V2: catalog stores the new schema mode on ViewInfo") { + val view = s"$catalog.$namespace.v2_schema_mode" + createView(view) + sql(s"ALTER VIEW $view WITH SCHEMA EVOLUTION") + val stored = viewCatalog.getStoredView(Array(namespace), "v2_schema_mode") + assert(stored.schemaMode == "EVOLUTION") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSetTblPropertiesSuite.scala new file mode 100644 index 000000000000..642d8d46b7fa --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSetTblPropertiesSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +/** + * The class contains tests for the `ALTER VIEW ... SET TBLPROPERTIES` command on V2 view + * catalogs (`AlterV2ViewSetPropertiesExec`). + */ +class AlterViewSetTblPropertiesSuite + extends command.AlterViewSetTblPropertiesSuiteBase with ViewCommandSuiteBase { + + test("V2: catalog stores the property on ViewInfo") { + val view = s"$catalog.$namespace.v2_set_view_info" + createView(view) + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('k' = 'v')") + val stored = viewCatalog.getStoredView(Array(namespace), "v2_set_view_info") + assert(stored.properties.get("k") == "v") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewUnsetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewUnsetTblPropertiesSuite.scala new file mode 100644 index 000000000000..0d7f13007e9f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewUnsetTblPropertiesSuite.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +class AlterViewUnsetTblPropertiesSuite + extends command.AlterViewUnsetTblPropertiesSuiteBase with ViewCommandSuiteBase { + + test("V2: unset removes the entry from the stored ViewInfo") { + val view = s"$catalog.$namespace.v2_unset_view_info" + createViewWithProps(view, "k" -> "v") + sql(s"ALTER VIEW $view UNSET TBLPROPERTIES ('k')") + val stored = viewCatalog.getStoredView(Array(namespace), "v2_unset_view_info") + assert(!stored.properties.containsKey("k")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala new file mode 100644 index 000000000000..44dee7e6da92 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, TableCatalog, TableInfo, TableSummary} +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.types.StructType + +class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase { + import testImplicits._ + + test("V2: CREATE VIEW propagates DEFAULT COLLATION onto the stored ViewInfo") { + val view = s"$catalog.$namespace.v2_create_collation" + withTable("spark_catalog.default.src_coll") { + Seq("a", "b").toDF("col").write.saveAsTable("spark_catalog.default.src_coll") + sql(s"CREATE VIEW $view DEFAULT COLLATION UTF8_BINARY AS " + + s"SELECT col FROM spark_catalog.default.src_coll") + val stored = viewCatalog.getStoredView(Array(namespace), "v2_create_collation") + assert(stored.properties().get(TableCatalog.PROP_COLLATION) == "UTF8_BINARY") + } + } + + test("V2: CREATE VIEW stamps PROP_OWNER on the stored TableInfo") { + val view = s"$catalog.$namespace.v2_create_owner" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + val stored = viewCatalog.getStoredView(Array(namespace), "v2_create_owner") + assert(stored.properties().containsKey(TableCatalog.PROP_OWNER)) + } + + test("V2: CREATE VIEW on a non-ViewCatalog catalog fails with MISSING_CATALOG_ABILITY.VIEWS") { + withSQLConf( + "spark.sql.catalog.no_view_cat" -> classOf[BasicInMemoryTableCatalog].getName) { + val ex = intercept[AnalysisException] { + sql("CREATE VIEW no_view_cat.default.v AS SELECT 1") + } + assert(ex.getCondition == "MISSING_CATALOG_ABILITY.VIEWS") + } + } + + test("V2: CREATE VIEW over a non-view table entry surfaces v1-parity errors") { + val ident = Identifier.of(Array(namespace), "v2_create_table_collide") + val tableInfo = new TableInfo.Builder() + .withSchema(new StructType().add("col", "string")) + .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) + .build() + viewCatalog.createTable(ident, tableInfo) + try { + // CREATE OR REPLACE VIEW must not silently destroy a non-view table -- v1 parity. + val replaceEx = intercept[AnalysisException] { + sql(s"CREATE OR REPLACE VIEW $catalog.$namespace.v2_create_table_collide AS " + + "SELECT 1 AS col") + } + assert(replaceEx.getCondition == "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE") + + // Plain CREATE VIEW over a table surfaces TABLE_OR_VIEW_ALREADY_EXISTS, matching v1. + val createEx = intercept[AnalysisException] { + sql(s"CREATE VIEW $catalog.$namespace.v2_create_table_collide AS SELECT 1 AS col") + } + assert(createEx.getCondition == "TABLE_OR_VIEW_ALREADY_EXISTS") + + // CREATE VIEW IF NOT EXISTS is a no-op -- the table entry is untouched. + sql(s"CREATE VIEW IF NOT EXISTS $catalog.$namespace.v2_create_table_collide AS " + + "SELECT 1 AS col") + val stored = viewCatalog.getStoredInfo(Array(namespace), "v2_create_table_collide") + assert(!stored.isInstanceOf[org.apache.spark.sql.connector.catalog.ViewInfo]) + } finally { + viewCatalog.dropTable(ident) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewColumnSuite.scala new file mode 100644 index 000000000000..ab0654d672f0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewColumnSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +class DescribeViewColumnSuite + extends command.DescribeViewColumnSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala new file mode 100644 index 000000000000..94f9a47e704b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +class DescribeViewSuite + extends command.DescribeViewSuiteBase with ViewCommandSuiteBase { + + test("V2: extended emits the v2-native `# Detailed View Information` header") { + val view = s"$catalog.$namespace.v2_desc_ext_header" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + val rows = sql(s"DESCRIBE TABLE EXTENDED $view").collect().map(_.getString(0)) + assert(rows.contains("# Detailed View Information"), + s"v2 extended describe should emit the View header; got:\n${rows.mkString("\n")}") + } + + test("V2: extended block includes view text and catalog name") { + val view = s"$catalog.$namespace.v2_desc_ext_body" + sql(s"CREATE VIEW $view AS SELECT 7 AS x") + val rows = sql(s"DESCRIBE TABLE EXTENDED $view").collect() + val pairs = rows.map(r => r.getString(0) -> r.getString(1)).toMap + assert(pairs.get("Catalog").contains(catalog)) + assert(pairs.get("View Text").exists(_.contains("SELECT 7 AS x"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala new file mode 100644 index 000000000000..4ea3dc911c39 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, TableInfo, TableSummary} +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.types.StructType + +class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase { + + test("V2: drop removes the entry from the catalog store") { + val view = s"$catalog.$namespace.v2_drop_remove" + sql(s"CREATE VIEW $view AS SELECT 1 AS x") + assert(viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_drop_remove"))) + sql(s"DROP VIEW $view") + assert(!viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_drop_remove"))) + } + + test("V2: DROP VIEW on a non-view table entry is rejected (v1-parity)") { + val ident = Identifier.of(Array(namespace), "v2_drop_table_collide") + val tableInfo = new TableInfo.Builder() + .withSchema(new StructType().add("x", "int")) + .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) + .build() + viewCatalog.createTable(ident, tableInfo) + try { + val ex = intercept[AnalysisException] { + sql(s"DROP VIEW $catalog.$namespace.v2_drop_table_collide") + } + assert(ex.getCondition.startsWith("EXPECT_VIEW_NOT_TABLE"), + s"expected EXPECT_VIEW_NOT_TABLE, got ${ex.getCondition}") + // The table entry is untouched. + assert(viewCatalog.tableExists(ident)) + } finally { + viewCatalog.dropTable(ident) + } + } + + test("V2: DROP VIEW on a non-ViewCatalog catalog fails") { + withSQLConf( + "spark.sql.catalog.no_view_drop_cat" -> classOf[BasicInMemoryTableCatalog].getName) { + val ex = intercept[AnalysisException] { + sql("DROP VIEW no_view_drop_cat.default.v") + } + // Resolution fails because the catalog cannot host views; the exact error condition + // depends on the resolver's not-found vs. capability-gate ordering. + assert(ex.getMessage.toLowerCase(java.util.Locale.ROOT).matches( + ".*(no such|not found|missing|not a view|cannot find|view).*"), + s"unexpected error: ${ex.getMessage}") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateViewSuite.scala new file mode 100644 index 000000000000..39bde3c7d58c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateViewSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +class ShowCreateViewSuite + extends command.ShowCreateViewSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewColumnsSuite.scala new file mode 100644 index 000000000000..92c90573cecd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewColumnsSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +class ShowViewColumnsSuite + extends command.ShowViewColumnsSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewPropertiesSuite.scala new file mode 100644 index 000000000000..44420167d3a7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewPropertiesSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +class ShowViewPropertiesSuite + extends command.ShowViewPropertiesSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala new file mode 100644 index 000000000000..ac4c4b475866 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, TableInfo, TableSummary} +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.types.StructType + +class ShowViewsSuite extends command.ShowViewsSuiteBase with ViewCommandSuiteBase { + + test("V2: SHOW VIEWS does not include non-view table entries") { + sql(s"CREATE VIEW $catalog.$namespace.v_v2_only_views AS SELECT 1 AS x") + val tableIdent = Identifier.of(Array(namespace), "t_v2_not_in_show_views") + viewCatalog.createTable( + tableIdent, + new TableInfo.Builder() + .withSchema(new StructType().add("x", "int")) + .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) + .build()) + try { + val rows = sql(s"SHOW VIEWS IN $catalog.$namespace").collect() + val names = rows.map(_.getString(1)).toSet + assert(names.contains("v_v2_only_views")) + assert(!names.contains("t_v2_not_in_show_views"), + s"non-view leaked into SHOW VIEWS: $names") + rows.foreach(r => assert(!r.getBoolean(2), + s"isTemporary must be false for v2 SHOW VIEWS: $r")) + } finally { + viewCatalog.dropTable(tableIdent) + } + } + + test("V2: SHOW VIEWS on a non-ViewCatalog catalog fails") { + withSQLConf( + "spark.sql.catalog.no_view_show_cat" -> classOf[BasicInMemoryTableCatalog].getName) { + val ex = intercept[AnalysisException] { + sql("SHOW VIEWS IN no_view_show_cat.default") + } + assert(ex.getCondition == "MISSING_CATALOG_ABILITY.VIEWS") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ViewCommandSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ViewCommandSuiteBase.scala new file mode 100644 index 000000000000..4dbbbc21afa9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ViewCommandSuiteBase.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.SparkConf +import org.apache.spark.sql.connector.catalog.InMemoryTableViewCatalog + +/** + * Settings for v2 view command test suites. Extends v2 [[CommandSuiteBase]] (so view tests + * inherit `checkLocation` and the standard v2 `test_catalog` configuration), and additionally + * wires `test_view_catalog` to [[InMemoryTableViewCatalog]] -- the catalog that the unified + * `*SuiteBase` view tests under `command/` target via the `$catalog` placeholder. + */ +trait ViewCommandSuiteBase extends CommandSuiteBase { + override def catalog: String = "test_view_catalog" + + override def sparkConf: SparkConf = super.sparkConf + .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableViewCatalog].getName) + + /** Helper: returns the configured `InMemoryTableViewCatalog`. */ + protected def viewCatalog: InMemoryTableViewCatalog = + spark.sessionState.catalogManager.catalog(catalog).asInstanceOf[InMemoryTableViewCatalog] +} From c649ef4c53c1c9a6b349c11778d0d72c4415ae4f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 09:44:10 +0000 Subject: [PATCH 04/17] [SPARK-56655][SQL] Address self-review on v2 view DDL/inspection PR Fixes from self-review of the v2 view DDL/inspection follow-up: - V1ViewInfo.builderFrom now bridges CatalogTable.collation/comment into the ViewInfo properties bag under PROP_COLLATION/PROP_COMMENT, so ApplyDefaultCollation.fetchDefaultCollation (which reads from properties) picks up the existing collation for v1 ALTER VIEW AS instead of returning None and skipping literal transformation. - DescribeRelationJsonCommand pattern-matches `info: V1ViewInfo` instead of unconditionally casting; non-session v2 views now fall through to the catch-all that surfaces describeAsJsonNotSupportedForV2TablesError, matching the v2 table path, instead of throwing ClassCastException. - ShowV2ViewPropertiesExec / DescribeV2ViewExec / ShowCreateV2ViewExec filter CatalogV2Util.TABLE_RESERVED_PROPERTIES (PROP_COMMENT, PROP_COLLATION, PROP_OWNER, PROP_TABLE_TYPE, ...) instead of just PROP_TABLE_TYPE, so SHOW TBLPROPERTIES / DESCRIBE EXTENDED on a v2 view no longer leak first-class fields that v1 hides. - AlterV2ViewSetPropertiesExec / UnsetPropertiesExec call CommandUtils.uncacheTableOrView before replaceView, matching v1 AlterTableSetPropertiesCommand's invalidateCachedTable. - Rewrite stale comments: ResolveSessionCatalog.ResolvedViewIdentifier, V2ViewInspectionExecs and DataSourceV2Strategy referenced `ResolvedPersistentView.viewInfo` and "tracked for a follow-up PR" text that no longer applied; minor grammar in DataSourceV2MetadataViewSuite. Co-authored-by: Isaac --- .../sql/connector/catalog/V1ViewInfo.scala | 7 +++++ .../analysis/ResolveSessionCatalog.scala | 10 +++---- .../command/DescribeRelationJsonCommand.scala | 10 ++++--- .../datasources/v2/AlterV2ViewExec.scala | 4 +++ .../datasources/v2/DataSourceV2Strategy.scala | 4 +-- .../v2/V2ViewInspectionExecs.scala | 30 +++++++++++-------- .../DataSourceV2MetadataViewSuite.scala | 4 +-- 7 files changed, 43 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala index 90a08e991c20..078768a68e3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala @@ -39,6 +39,13 @@ private[sql] object V1ViewInfo { val builder = new ViewInfo.Builder() builder.withSchema(v1Table.schema) builder.withProperties(v1Table.properties.asJava) + // v1 stores collation / comment in typed `CatalogTable` fields rather than in `properties`, + // but consumers reading off [[ViewInfo]] (`ApplyDefaultCollation.fetchDefaultCollation`, + // `ShowCreateV2ViewExec`, etc.) expect them under `PROP_COLLATION` / `PROP_COMMENT`. Bridge + // them through the typed setters so the v2 surface sees the same view metadata regardless + // of which catalog produced it. + v1Table.collation.foreach(builder.withCollation) + v1Table.comment.foreach(builder.withComment) // ViewInfo requires a non-null queryText; v1 views always have one, but defend against // an old/corrupt CatalogTable with `viewText = None` by falling back to an empty string. builder.withQueryText(v1Table.viewText.getOrElse("")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 0dbcc3f51756..f5c881bad760 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -796,11 +796,11 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) object ResolvedViewIdentifier { // Only matches session-catalog persistent views. Non-session-catalog persistent views - // (produced for `MetadataTable`) fall through; `AlterViewAs` is picked up by the v2 - // strategy, and the remaining view DDL / inspection plans (SET/UNSET TBLPROPERTIES, - // ALTER VIEW ... WITH SCHEMA, RENAME TO, SHOW CREATE TABLE, SHOW TBLPROPERTIES, SHOW - // COLUMNS, DESCRIBE [COLUMN]) are rejected with `UNSUPPORTED_FEATURE.TABLE_OPERATION` by - // dedicated v2 strategy cases -- tracked for a follow-up PR (SPARK-52729). + // (produced for `MetadataTable`) fall through and are picked up by dedicated v2 strategy + // cases in `DataSourceV2Strategy` -- AlterViewAs, SET/UNSET TBLPROPERTIES, ALTER VIEW ... + // WITH SCHEMA, RENAME TO, SHOW CREATE TABLE, SHOW TBLPROPERTIES, SHOW COLUMNS, DESCRIBE + // [COLUMN] all dispatch to v2 view execs that consume `ResolvedPersistentView.info` + // directly. def unapply(resolved: LogicalPlan): Option[TableIdentifier] = resolved match { case ResolvedPersistentView(catalog, ident, _) if isSessionCatalog(catalog) => Some(ident.asTableIdentifier.copy(catalog = Some(catalog.name))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala index 3ca07f9f79aa..562aab876e70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala @@ -67,13 +67,15 @@ case class DescribeRelationJsonCommand( describeColsJson(v.metadata.schema, jsonMap) describeFormattedTableInfoJson(v.metadata, jsonMap) - case v: ResolvedPersistentView => + case v @ ResolvedPersistentView(_, _, v1Info: V1ViewInfo) => + // Only session-catalog (v1) views are supported here. Non-session v2 views fall + // through to the catch-all below and surface + // `describeAsJsonNotSupportedForV2TablesError`, mirroring the v2 table path -- there + // is no `DESC ... AS JSON` exec on the v2 view side yet. if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(v.identifier.name()) } - // v1 paths reach here only for session-catalog views (`V1ViewInfo`); v2 views go - // through `DescribeV2ViewExec` in the v2 strategy. - val metadata = v.info.asInstanceOf[V1ViewInfo].v1Table + val metadata = v1Info.v1Table describeIdentifier(v.identifier.toQualifiedNameParts(v.catalog), jsonMap) describeColsJson(metadata.schema, jsonMap) describeFormattedTableInfoJson(metadata, jsonMap) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala index 3938c08b0179..22d7a4bc522a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala @@ -129,6 +129,9 @@ case class AlterV2ViewSetPropertiesExec( val info = V2ViewMetadataMutation.builderFrom(existingView) .withProperties(merged.asJava) .build() + // Match v1 `AlterTableSetPropertiesCommand`'s `invalidateCachedTable` so cached query + // plans referencing the view drop their stale entries. + CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) catalog.replaceView(identifier, info) Seq.empty } @@ -154,6 +157,7 @@ case class AlterV2ViewUnsetPropertiesExec( val info = V2ViewMetadataMutation.builderFrom(existingView) .withProperties(remaining.asJava) .build() + CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) catalog.replaceView(identifier, info) Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 6a00af8be9b5..f51f9159ecf0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -326,8 +326,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // View DDL / inspection on a non-session v2 catalog that the v1 rewrite in // `ResolveSessionCatalog` can't handle (its `ResolvedViewIdentifier` matcher is gated on // `isSessionCatalog`). Routed to dedicated v2 execs that read the typed `ViewInfo` - // resolved at analysis time directly from `ResolvedPersistentView.viewInfo` -- no - // re-loading at exec time. + // resolved at analysis time directly from `ResolvedPersistentView.info` -- no re-loading + // at exec time. case SetViewProperties(rpv @ ResolvedPersistentView(catalog, ident, _), props) => AlterV2ViewSetPropertiesExec( catalog.asInstanceOf[ViewCatalog], ident, rpv.info, props) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala index f96317ff7e12..ad5659c6bc8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala @@ -24,12 +24,12 @@ import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIfNeeded, ResolveDefaultColumns} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumnsUtils.CURRENT_DEFAULT_COLUMN_METADATA_KEY -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, ViewInfo} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog, ViewInfo} import org.apache.spark.sql.errors.QueryCompilationErrors /** * Read-side v2 view execs. Each receives the typed [[ViewInfo]] resolved at analysis time - * (carried on `ResolvedPersistentView.viewInfo`) and formats output rows directly from it -- + * (carried on `ResolvedPersistentView.info`) and formats output rows directly from it -- * matching the way v2 table inspection execs (e.g. `ShowCreateTableExec`, `DescribeTableExec`) * consume the [[org.apache.spark.sql.connector.catalog.Table]] attached to `ResolvedTable`. * @@ -82,14 +82,12 @@ case class ShowCreateV2ViewExec( } private def showViewProperties(builder: StringBuilder): Unit = { - // Drop the reserved keys that already appear as dedicated DDL clauses (PROP_COMMENT / - // PROP_COLLATION) and the auto-injected `table_type`. Anything left is user TBLPROPERTIES. - val reserved = Set( - TableCatalog.PROP_COMMENT, - TableCatalog.PROP_COLLATION, - TableCatalog.PROP_TABLE_TYPE) + // Drop the reserved keys that either already appear as dedicated DDL clauses + // (PROP_COMMENT / PROP_COLLATION) or are otherwise managed outside user TBLPROPERTIES + // (PROP_OWNER, PROP_TABLE_TYPE, etc.). Mirrors the v1 SHOW CREATE TABLE filter, which + // hides the same first-class fields from the rendered TBLPROPERTIES clause. val viewProps = viewInfo.properties.asScala - .filter { case (k, _) => !reserved.contains(k) } + .filter { case (k, _) => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(k) } if (viewProps.nonEmpty) { val props = viewProps.toSeq.sortBy(_._1).map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" @@ -101,8 +99,11 @@ case class ShowCreateV2ViewExec( /** * Physical plan node for SHOW TBLPROPERTIES on a v2 view. Returns the user-facing properties - * from [[ViewInfo#properties]] -- the auto-injected `table_type` is filtered out so users see - * only what they (or the catalog) explicitly set. + * from [[ViewInfo#properties]] -- reserved first-class keys (PROP_COMMENT, PROP_COLLATION, + * PROP_OWNER, PROP_TABLE_TYPE, ...) are filtered out so users see only what they (or the + * catalog) explicitly set, matching v1 `SHOW TBLPROPERTIES` on a session-catalog view (which + * hides these because v1 stores them in typed `CatalogTable` fields rather than `properties`). + * A directly-requested reserved key still returns its value so users can ask for it by name. */ case class ShowV2ViewPropertiesExec( output: Seq[Attribute], @@ -124,7 +125,7 @@ case class ShowV2ViewPropertiesExec( } case None => redacted - .filter { case (k, _) => k != TableCatalog.PROP_TABLE_TYPE } + .filter { case (k, _) => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(k) } .toSeq.sortBy(_._1).map(p => toCatalystRow(p._1, p._2)) } } @@ -186,8 +187,11 @@ case class DescribeV2ViewExec( result += toCatalystRow( "View Query Output Columns", queryColumns.mkString("[", ", ", "]"), "") } + // Filter the same reserved set as `ShowV2ViewPropertiesExec` so the EXTENDED + // `Properties` row mirrors `SHOW TBLPROPERTIES` and matches v1 (which hides these + // first-class fields because they live in typed `CatalogTable` fields). val userProps = viewInfo.properties.asScala - .filter { case (k, _) => k != TableCatalog.PROP_TABLE_TYPE } + .filter { case (k, _) => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(k) } if (userProps.nonEmpty) { val props = conf.redactOptions(userProps.toMap).toSeq.sortBy(_._1).map { case (k, v) => s"$k=$v" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala index 79d003c54a69..25879deb02c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala @@ -51,8 +51,8 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { withTable("spark_catalog.default.t") { Seq("a", "b").toDF("col").write.saveAsTable("spark_catalog.default.t") // view_catalog.ansi.test_view stores view.sqlConfig.spark.sql.ansi.enabled=true; - // view_catalog.non_ansi.test_view stores it =false. The view body does - // `col::int` which errors in ANSI mode and yields NULL in non-ANSI mode. + // view_catalog.non_ansi.test_view stores the same key with value `false`. The view body + // does `col::int` which errors in ANSI mode and yields NULL in non-ANSI mode. intercept[Exception](spark.table("view_catalog.ansi.test_view").collect()) checkAnswer(spark.table("view_catalog.non_ansi.test_view"), Row("b", null)) } From b5cacaa1e1e85b2f0d651d3df0633f625c2a350f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 11:06:59 +0000 Subject: [PATCH 05/17] [SPARK-56655][SQL] Propagate namespace default collation through ALTER VIEW AS Address Pass-A finding from the v2 view DDL/inspection self-review: when an existing view has no `PROP_COLLATION` and its namespace has a default collation, `ALTER VIEW AS` now folds the namespace default into the persisted view metadata, on both v1 and v2 paths. Three pieces: 1. v1: plumb the analysis-time collation through `AlterViewAsCommand`. `ResolveSessionCatalog` reads `ResolvedPersistentView.info.properties`'s `PROP_COLLATION` (post-`ApplyDefaultCollation` rewrite) and passes it to `AlterViewAsCommand`. `alterPermanentView` writes it onto the persisted `CatalogTable.collation` so subsequent reads pick it up via `AnalysisContext.collation`. Without this fix, the analyzed plan's literals were collated correctly but the persisted typed `collation` field was lost because `alterPermanentView` reloaded `viewMeta` from the catalog and used `viewMeta.copy(...)` without touching `collation`. 2. v2: generalize `ApplyDefaultCollation.resolveDefaultCollation`'s `AlterViewAs` case to fire for any `ResolvedPersistentView` whose `info.properties.PROP_COLLATION` is empty -- both `V1ViewInfo` (existing v1 path) and plain `ViewInfo` (v2 path). The v2 branch rebuilds the `ViewInfo` with the namespace default via a shared `CatalogV2Util.viewInfoBuilderFrom` helper (extracted from the existing `V2ViewMetadataMutation.builderFrom` and reused). Gate the rewrite on a non-empty namespace default to avoid an infinite resolution loop -- `ViewInfo` / `V1ViewInfo` are not case classes, so a copy with identical field values reads as a different reference under fast-equals. 3. Tests: extend `InMemoryTableViewCatalog` with a minimal `SupportsNamespaces` impl so v2 view tests can exercise namespace metadata; add a v1 scenario-C test in `DefaultCollationTestSuite` and a matching v2 test in `command/v2/AlterViewAsSuite`. Co-authored-by: Isaac --- .../analysis/ApplyDefaultCollation.scala | 44 +++++++--- .../sql/connector/catalog/CatalogV2Util.scala | 21 +++++ .../catalog/InMemoryTableViewCatalog.scala | 86 ++++++++++++++++++- .../analysis/ResolveSessionCatalog.scala | 18 +++- .../spark/sql/execution/command/views.scala | 18 +++- .../datasources/v2/AlterV2ViewExec.scala | 32 +------ .../collation/DefaultCollationTestSuite.scala | 23 +++++ .../command/v2/AlterViewAsSuite.scala | 29 +++++++ 8 files changed, 222 insertions(+), 49 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala index 640faf3fbe1b..375a7d3d6e9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.types.DataTypeUtils.{areSameBaseType, isDefaultStringCharOrVarcharType, replaceDefaultStringCharAndVarcharTypes} import org.apache.spark.sql.catalyst.util.CharVarcharUtils -import org.apache.spark.sql.connector.catalog.{SupportsNamespaces, TableCatalog, V1ViewInfo} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog, V1ViewInfo, ViewInfo} import org.apache.spark.sql.types.{DataType, StringHelper, StringType} /** @@ -207,19 +207,37 @@ object ApplyDefaultCollation extends Rule[LogicalPlan] { newCreateView // We match against ResolvedPersistentView because temporary views don't have a - // schema/catalog. Only the v1 (session-catalog) form is rewritten here -- it carries - // the underlying `CatalogTable` via `V1ViewInfo`, so we can update `collation` and - // re-wrap. v2 view paths consume `info.properties` directly and are not affected by - // this rewrite. - case alterViewAs@AlterViewAs(resolvedPersistentView@ResolvedPersistentView( - catalog: SupportsNamespaces, identifier, v1Info: V1ViewInfo), _, _, _, _) - if v1Info.v1Table.collation.isEmpty => - val newCollation = getCollationFromSchemaMetadata(catalog, identifier.namespace()) - val newV1Table = v1Info.v1Table.copy(collation = newCollation) - val newResolvedPersistentView = resolvedPersistentView.copy( - info = new V1ViewInfo(newV1Table)) + // schema/catalog. The rewrite covers both v1 (session-catalog, [[V1ViewInfo]]) and + // non-session v2 views: when the existing view has no `PROP_COLLATION`, fold the + // namespace's default collation into the resolved `ViewInfo`. For v1, `V1ViewInfo` + // is rebuilt around a `CatalogTable` whose typed `collation` field holds the new + // value; `V1ViewInfo.builderFrom` bridges that into the v2 `properties()` bag, so + // downstream consumers (`fetchDefaultCollation`, `AlterV2ViewExec`'s + // `existingProp(PROP_COLLATION)`) see it under either surface. For v2, we rebuild the + // existing `ViewInfo` with `PROP_COLLATION` set so the same downstream consumers see + // it on the regular `info.properties` path. + case alterViewAs @ AlterViewAs(rpv @ ResolvedPersistentView( + catalog: SupportsNamespaces, identifier, info), _, _, _, _) + if Option(info.properties.get(TableCatalog.PROP_COLLATION)).isEmpty && + getCollationFromSchemaMetadata(catalog, identifier.namespace()).nonEmpty => + // Only rewrite when the namespace actually supplies a default. [[ViewInfo]] / + // [[V1ViewInfo]] are non-case classes, so a copy with structurally-identical fields + // still reads as a different reference -- if we rewrote unconditionally, the + // resolution batch would see the plan change every iteration and never reach + // fixed point. + val newCollation = + getCollationFromSchemaMetadata(catalog, identifier.namespace()).get + val newInfo: ViewInfo = info match { + case v1Info: V1ViewInfo => + new V1ViewInfo(v1Info.v1Table.copy(collation = Some(newCollation))) + case _ => + CatalogV2Util.viewInfoBuilderFrom(info) + .withCollation(newCollation) + .build() + } + val newRpv = rpv.copy(info = newInfo) val newAlterViewAs = CurrentOrigin.withOrigin(alterViewAs.origin) { - alterViewAs.copy(child = newResolvedPersistentView) + alterViewAs.copy(child = newRpv) } newAlterViewAs.copyTagsFrom(alterViewAs) newAlterViewAs diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index b29d0b3eabe5..a7c5cf5e5431 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -533,6 +533,27 @@ private[sql] object CatalogV2Util { catalog.name().equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME) } + /** + * Construct a [[ViewInfo.Builder]] seeded from an existing view's metadata. Used by ALTER + * VIEW execs (SET / UNSET TBLPROPERTIES, ALTER VIEW ... WITH SCHEMA BINDING) and by + * `ApplyDefaultCollation` -- override the one field that changes, then `build` to produce + * the replacement payload for [[ViewCatalog#replaceView]]. Every other field flows through + * unchanged so a metadata-only mutation does not perturb the view body. + */ + def viewInfoBuilderFrom(existing: ViewInfo): ViewInfo.Builder = { + val builder = new ViewInfo.Builder() + builder + .withSchema(existing.schema) + .withProperties(existing.properties) + .withQueryText(existing.queryText) + .withSqlConfigs(existing.sqlConfigs) + .withCurrentNamespace(existing.currentNamespace) + .withQueryColumnNames(existing.queryColumnNames) + Option(existing.currentCatalog).foreach(builder.withCurrentCatalog) + Option(existing.schemaMode).foreach(builder.withSchemaMode) + builder + } + def convertTableProperties(t: TableSpec): Map[String, String] = { val props = convertTableProperties( t.properties, t.options, t.serde, t.location, t.comment, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala index 28355ba7abe4..a3506938dea7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableViewCatalog.scala @@ -17,21 +17,29 @@ package org.apache.spark.sql.connector.catalog +import java.util import java.util.concurrent.ConcurrentHashMap -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, NoSuchViewException, TableAlreadyExistsException, ViewAlreadyExistsException} +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, NoSuchViewException, TableAlreadyExistsException, ViewAlreadyExistsException} import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * An in-memory [[TableViewCatalog]] for tests. Tables and views share a single keyspace per * the [[TableViewCatalog]] contract; the stored value's runtime type ([[TableInfo]] vs - * [[ViewInfo]]) is the kind discriminator. Suitable for any test suite that wants to exercise + * [[ViewInfo]]) is the kind discriminator. Also implements [[SupportsNamespaces]] with a + * minimal namespace store, so analyzer rules that read namespace metadata (e.g. + * `ApplyDefaultCollation` consulting `loadNamespaceMetadata` for `PROP_COLLATION`) work + * uniformly with the v1 session catalog. Suitable for any test suite that wants to exercise * v2 view DDL or inspection commands against a non-session catalog. */ -class InMemoryTableViewCatalog extends TableViewCatalog { +class InMemoryTableViewCatalog extends TableViewCatalog with SupportsNamespaces { private val store = new ConcurrentHashMap[(Seq[String], String), TableInfo]() + private val namespaces = + new ConcurrentHashMap[Seq[String], util.Map[String, String]]() override def loadTableOrView(ident: Identifier): Table = { val key = (ident.namespace().toSeq, ident.name()) @@ -136,6 +144,78 @@ class InMemoryTableViewCatalog extends TableViewCatalog { store.remove(oldKey) } + // ----- SupportsNamespaces ----------------------------------------------------------- + + // A namespace exists if it was explicitly created or if any stored entry sits under it. + private def implicitNamespaces: Set[Seq[String]] = + store.keySet.asScala.iterator.map(_._1).toSet + + override def listNamespaces(): Array[Array[String]] = { + val all = (namespaces.keySet.asScala ++ implicitNamespaces).toSet + all.iterator.filter(_.nonEmpty).map(ns => Array(ns.head)).toArray.distinct + } + + override def listNamespaces(parent: Array[String]): Array[Array[String]] = { + val parentSeq = parent.toSeq + val all = (namespaces.keySet.asScala ++ implicitNamespaces).toSet + all.iterator + .filter(_.size > parentSeq.size) + .filter(_.startsWith(parentSeq)) + .map(_.take(parentSeq.size + 1).toArray) + .toArray + .distinct + } + + override def namespaceExists(namespace: Array[String]): Boolean = { + val ns = namespace.toSeq + namespaces.containsKey(ns) || implicitNamespaces.exists(_.startsWith(ns)) + } + + override def loadNamespaceMetadata(namespace: Array[String]): util.Map[String, String] = { + val ns = namespace.toSeq + Option(namespaces.get(ns)) match { + case Some(metadata) => metadata + case _ if namespaceExists(namespace) => util.Collections.emptyMap[String, String] + case _ => throw new NoSuchNamespaceException(name() +: namespace) + } + } + + override def createNamespace( + namespace: Array[String], + metadata: util.Map[String, String]): Unit = { + val ns = namespace.toSeq + if (namespaces.putIfAbsent(ns, new util.HashMap[String, String](metadata)) != null) { + throw new NamespaceAlreadyExistsException(namespace) + } + } + + override def alterNamespace( + namespace: Array[String], + changes: NamespaceChange*): Unit = { + val ns = namespace.toSeq + val current = Option(namespaces.get(ns)).getOrElse { + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException(name() +: namespace) + } + new util.HashMap[String, String]() + } + val updated = CatalogV2Util.applyNamespaceChanges(current, changes.toSeq) + namespaces.put(ns, updated) + } + + override def dropNamespace(namespace: Array[String], cascade: Boolean): Boolean = { + val ns = namespace.toSeq + if (!cascade && implicitNamespaces.exists(_.startsWith(ns))) { + throw new org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException( + name() +: namespace) + } + if (cascade) { + val keysToRemove = store.keySet.asScala.filter(_._1.startsWith(ns)).toSeq + keysToRemove.foreach(store.remove) + } + namespaces.remove(ns) != null || implicitNamespaces.exists(_.startsWith(ns)) + } + // Test-only accessors -------------------------------------------------------------- /** Returns the stored entry (table or view) for the identifier, or throws if missing. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index f5c881bad760..2f03498d4ce2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, CharVarcharUtils, ResolveDefaultColumns => DefaultCols} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ -import org.apache.spark.sql.connector.catalog.{CatalogExtension, CatalogManager, CatalogPlugin, CatalogV2Util, LookupCatalog, SupportsNamespaces, V1Table, ViewCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogExtension, CatalogManager, CatalogPlugin, CatalogV2Util, LookupCatalog, SupportsNamespaces, TableCatalog, V1Table, ViewCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.command._ @@ -535,8 +535,20 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // The final `_, _` are AlterViewAs.isAnalyzed and referredTempFunctions. We drop both: // AlterViewAsCommand is a separate AnalysisOnlyCommand and gets its own markAsAnalyzed pass // from HandleSpecialCommand after this rewrite. - case AlterViewAs(ResolvedViewIdentifier(ident), originalText, query, _, _) => - AlterViewAsCommand(ident, originalText, query) + case alterViewAs @ AlterViewAs( + ResolvedViewIdentifier(ident), originalText, query, _, _) => + // For session-catalog persistent views, pick up the analysis-time collation off the + // resolved `ViewInfo` -- `ApplyDefaultCollation` rewrites that property to fill the + // namespace default when the existing view had none, and `alterPermanentView` wants + // the post-rewrite value so the persisted `CatalogTable.collation` matches the + // collated literal types in the analyzed plan. Temp views don't carry a `ViewInfo`, + // so they pass through without a collation override. + val collation = alterViewAs.child match { + case rpv: ResolvedPersistentView => + Option(rpv.info.properties.get(TableCatalog.PROP_COLLATION)) + case _ => None + } + AlterViewAsCommand(ident, originalText, query, collation = collation) case AlterViewSchemaBinding(ResolvedViewIdentifier(ident), viewSchemaMode) => AlterViewSchemaBindingCommand(ident, viewSchemaMode) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 994c7836f9dd..62f271c05591 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -235,7 +235,15 @@ case class AlterViewAsCommand( originalText: String, query: LogicalPlan, isAnalyzed: Boolean = false, - referredTempFunctions: Seq[String] = Seq.empty) + referredTempFunctions: Seq[String] = Seq.empty, + // Analysis-time collation for the resolved view (after `ApplyDefaultCollation` has had a + // chance to fill the namespace default into a previously-empty `CatalogTable.collation`). + // `ResolveSessionCatalog` populates this from `ResolvedPersistentView.info.properties`'s + // `PROP_COLLATION`; only `alterPermanentView` consumes it. `None` means "no collation + // specified at analysis time" -- `alterPermanentView` then preserves the existing + // `CatalogTable.collation` so external constructors that omit this argument retain the + // pre-PR behavior. + collation: Option[String] = None) extends RunnableCommand with AnalysisOnlyCommand with CTEInChildren { import ViewHelper._ @@ -310,7 +318,13 @@ case class AlterViewAsCommand( schema = newSchema, properties = newProperties, viewOriginalText = Some(originalText), - viewText = Some(originalText)) + viewText = Some(originalText), + // Prefer the analysis-time collation -- `ApplyDefaultCollation` may have filled the + // namespace's default `PROP_COLLATION` into a previously-empty value, and we want that + // to be persisted alongside the matching collated literal types in `newSchema`. Fall + // back to the existing typed field for backward compat with callers that don't pass + // `collation`. + collation = collation.orElse(viewMeta.collation)) session.sessionState.catalog.alterTable(updatedViewMeta) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala index 22d7a4bc522a..9c6debe23d38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala @@ -23,33 +23,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ViewSchemaMode} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, ViewCatalog, ViewInfo} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog, ViewCatalog, ViewInfo} import org.apache.spark.sql.execution.command.CommandUtils -private[v2] object V2ViewMetadataMutation { - /** - * Construct a [[ViewInfo.Builder]] seeded from an existing view's metadata. Mutating execs - * (SET / UNSET TBLPROPERTIES, ALTER VIEW ... WITH SCHEMA BINDING) start here, override the - * one field they're changing, and call [[ViewInfo.Builder#build]] to produce the replacement - * payload for [[ViewCatalog#replaceView]]. Everything else -- columns, queryText, captured - * resolution context, captured SQL configs, queryColumnNames -- flows through unchanged so - * a metadata-only mutation does not perturb the view body. - */ - def builderFrom(existing: ViewInfo): ViewInfo.Builder = { - val builder = new ViewInfo.Builder() - builder - .withSchema(existing.schema) - .withProperties(existing.properties) - .withQueryText(existing.queryText) - .withSqlConfigs(existing.sqlConfigs) - .withCurrentNamespace(existing.currentNamespace) - .withQueryColumnNames(existing.queryColumnNames) - Option(existing.currentCatalog).foreach(builder.withCurrentCatalog) - Option(existing.schemaMode).foreach(builder.withSchemaMode) - builder - } -} - /** * Shared bits for the v2 ALTER VIEW ... AS exec. The replacement [[ViewInfo]] is constructed by * [[V2ViewPreparation.buildViewInfo]]; the existing view's payload is provided at analysis time @@ -126,7 +102,7 @@ case class AlterV2ViewSetPropertiesExec( override protected def run(): Seq[InternalRow] = { val merged = existingView.properties.asScala.toMap ++ properties - val info = V2ViewMetadataMutation.builderFrom(existingView) + val info = CatalogV2Util.viewInfoBuilderFrom(existingView) .withProperties(merged.asJava) .build() // Match v1 `AlterTableSetPropertiesCommand`'s `invalidateCachedTable` so cached query @@ -154,7 +130,7 @@ case class AlterV2ViewUnsetPropertiesExec( override protected def run(): Seq[InternalRow] = { val remaining = existingView.properties.asScala.toMap -- propertyKeys - val info = V2ViewMetadataMutation.builderFrom(existingView) + val info = CatalogV2Util.viewInfoBuilderFrom(existingView) .withProperties(remaining.asJava) .build() CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) @@ -178,7 +154,7 @@ case class AlterV2ViewSchemaBindingExec( override def output: Seq[org.apache.spark.sql.catalyst.expressions.Attribute] = Seq.empty override protected def run(): Seq[InternalRow] = { - val info = V2ViewMetadataMutation.builderFrom(existingView) + val info = CatalogV2Util.viewInfoBuilderFrom(existingView) .withSchemaMode(viewSchemaMode.toString) .build() CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala index 670f4540a810..29ee48e02919 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala @@ -1556,6 +1556,29 @@ abstract class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { } } + testString("ALTER VIEW AS picks up the namespace's default collation when the existing " + + "view has none") { + _ => + // The view is created in a schema with no default collation, so the stored + // `CatalogTable.collation` is `None`. After the schema gains a default, the next + // `ALTER VIEW AS` must fold that default into both the analyzed plan's literal types + // (so `assertTableColumnCollation` sees it on read) and the persisted CatalogTable so + // the AnalysisContext fallback fires on every subsequent read. + withDatabase(testSchema) { + sql(s"CREATE SCHEMA $testSchema") + sql(s"USE $testSchema") + withView(testView) { + sql(s"CREATE VIEW $testView AS SELECT 'a' AS c1") + assertTableColumnCollation(testView, "c1", "UTF8_BINARY") + + sql(s"ALTER SCHEMA $testSchema DEFAULT COLLATION UTF8_LCASE") + sql(s"ALTER VIEW $testView AS SELECT 'x' AS c1, 'y' AS c2") + assertTableColumnCollation(testView, "c1", "UTF8_LCASE") + assertTableColumnCollation(testView, "c2", "UTF8_LCASE") + } + } + } + private def testCreateViewWithSchemaLevelCollation( dataType: String, schemaDefaultCollation: String, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala index 08953bfa96d1..2ed112c3b22c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewAsSuite.scala @@ -21,9 +21,38 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, TableCatalog} import org.apache.spark.sql.execution.command import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StringType class AlterViewAsSuite extends command.AlterViewAsSuiteBase with ViewCommandSuiteBase { + test("V2: ALTER VIEW AS picks up the namespace's default collation when the existing view " + + "has none") { + // Create the namespace with no default collation; create a view in it (PROP_COLLATION + // unset). Then set the namespace default and ALTER VIEW AS -- the new ViewInfo must end + // up with PROP_COLLATION = UTF8_LCASE (so v1Table.toCatalogTable's `collation` field is + // set, and view-read time picks up UTF8_LCASE via AnalysisContext.collation). + withSQLConf(SQLConf.SCHEMA_LEVEL_COLLATIONS_ENABLED.key -> "true") { + val viewName = "v2_alter_collation_inherit" + val view = s"$catalog.$namespace.$viewName" + sql(s"CREATE VIEW $view AS SELECT 'a' AS c1") + assert(Option(viewCatalog + .getStoredView(Array(namespace), viewName) + .properties() + .get(TableCatalog.PROP_COLLATION)) + .isEmpty) + + sql(s"ALTER NAMESPACE $catalog.$namespace DEFAULT COLLATION UTF8_LCASE") + sql(s"ALTER VIEW $view AS SELECT 'x' AS c1, 'y' AS c2") + + val stored = viewCatalog.getStoredView(Array(namespace), viewName) + assert(stored.properties().get(TableCatalog.PROP_COLLATION) == "UTF8_LCASE") + // Read-time the view body's literal types reflect the inherited collation. + val df = spark.table(view) + assert(df.schema("c1").dataType === StringType("UTF8_LCASE")) + assert(df.schema("c2").dataType === StringType("UTF8_LCASE")) + } + } + test("V2: ALTER VIEW preserves PROP_OWNER (v1-parity)") { val view = s"$catalog.$namespace.v2_alter_keep_owner" sql(s"CREATE VIEW $view AS SELECT 1 AS x") From 75375cfaf12e7973034bddd7189a618328f601a7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 12:01:34 +0000 Subject: [PATCH 06/17] [SPARK-56655][SQL] Self-review fixes: re-cache v2 view after RENAME, doc V1ViewInfo properties caveat, tighten tests - RenameV2ViewExec now captures the cached storage level before uncaching and re-instates the cache at the new identifier afterwards, mirroring v1 AlterTableRenameCommand and v2 RenameTableExec. Without it a user-cached view on a v2 catalog silently lost its cache after RENAME. - V1ViewInfo Scaladoc calls out that the inherited properties() bag mixes user TBLPROPERTIES with v1-internal storage keys (view.sqlConfig.*, view.catalogAndNamespace.*, view.query.out.*, view.schemaMode), and consumers should prefer typed accessors for the v1-internal fields. - ShowCreateViewSuiteBase "renders the column list" test now uses distinctive column names so the contains() assertions aren't satisfied by tokens elsewhere in the rendered DDL. - AlterViewRenameSuiteBase adds a "rename re-caches a previously cached view" base test that runs against both v1 and v2 paths and pins the new RenameV2ViewExec cache-restore behavior. - AlterViewRenameSuite (v2 leaf) adds a cross-namespace rename test exercising RenameV2ViewExec's non-empty-namespace branch. --- .../sql/connector/catalog/V1ViewInfo.scala | 11 ++++++++++ .../datasources/v2/AlterV2ViewExec.scala | 21 ++++++++++++++++++ .../command/AlterViewRenameSuiteBase.scala | 22 +++++++++++++++++++ .../command/ShowCreateViewSuiteBase.scala | 8 ++++--- .../command/v2/AlterViewRenameSuite.scala | 17 ++++++++++++++ 5 files changed, 76 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala index 078768a68e3d..e18fe52385a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1ViewInfo.scala @@ -26,6 +26,17 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable * the v2 [[Table]] surface. Holds the original [[CatalogTable]] in [[v1Table]] for v1-only * paths that need the full v1 metadata representation (e.g. `DescribeTableCommand`, * `ShowCreateTableCommand`, anything that calls `CatalogTable#toLinkedHashMap`). + * + * Note on `properties()`: the inherited [[ViewInfo#properties]] bag is built from the entire + * `v1Table.properties` map, which intermixes user TBLPROPERTIES with v1-internal storage keys + * (`view.sqlConfig.*`, `view.catalogAndNamespace.*`, `view.query.out.*`, `view.schemaMode`). + * v2 view inspection / SET execs (`ShowV2ViewPropertiesExec`, `AlterV2ViewSetPropertiesExec`, + * etc.) never see a `V1ViewInfo` -- `ResolveSessionCatalog` rewrites session-catalog views to + * v1 commands first -- so the bag stays internal to v1-only paths. Consumers that do receive + * a `V1ViewInfo` should prefer the typed accessors ([[ViewInfo#sqlConfigs]], + * [[ViewInfo#currentNamespace]], [[ViewInfo#currentCatalog]], [[ViewInfo#queryColumnNames]], + * [[ViewInfo#schemaMode]]) for the v1-internal fields rather than scraping `properties()` for + * them. */ private[sql] class V1ViewInfo(val v1Table: CatalogTable) extends ViewInfo(V1ViewInfo.builderFrom(v1Table)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala index 9c6debe23d38..417234a01733 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ViewSchemaMod import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog, ViewCatalog, ViewInfo} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.{IdentifierHelper, MultipartIdentifierHelper} import org.apache.spark.sql.execution.command.CommandUtils /** @@ -168,6 +169,11 @@ case class AlterV2ViewSchemaBindingExec( * [[ViewCatalog#renameView]]; if the source view is missing or has been replaced with a non-view * table between analysis and exec, the catalog throws `NoSuchViewException` and the error * propagates. + * + * If the view was cached at the old identifier, the cache entry is captured before the rename + * and re-instated at the new identifier afterwards -- matches v1 `AlterTableRenameCommand` and + * v2 `RenameTableExec`, so users on a v2 view catalog who explicitly cached a view do not + * silently lose the cache after a rename. */ case class RenameV2ViewExec( catalog: ViewCatalog, @@ -182,9 +188,24 @@ case class RenameV2ViewExec( val qualifiedNewIdent = if (newIdent.namespace.isEmpty) { Identifier.of(oldIdent.namespace, newIdent.name) } else newIdent + + // Capture the old view's storage level before uncaching, mirroring v1 + // `AlterTableRenameCommand`. Resolving the old identifier via `session.table` runs through + // view-text expansion so the cache lookup keys off the same plan that was originally + // cached via `CACHE TABLE `. + val oldQualified = (catalog.name() +: oldIdent.asMultipartIdentifier).quoted + val optStorageLevel = session.sharedState.cacheManager + .lookupCachedData(session.table(oldQualified)) + .map(_.cachedRepresentation.cacheBuilder.storageLevel) + CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, oldIdent)) catalog.invalidateView(oldIdent) catalog.renameView(oldIdent, qualifiedNewIdent) + + optStorageLevel.foreach { storageLevel => + val newQualified = (catalog.name() +: qualifiedNewIdent.asMultipartIdentifier).quoted + session.catalog.cacheTable(newQualified, storageLevel) + } Seq.empty } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala index b9f694c92fe3..1620425c2abd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala @@ -83,4 +83,26 @@ trait AlterViewRenameSuiteBase extends QueryTest with DDLCommandTestUtils { assert(ex.getCondition.startsWith("EXPECT_TABLE_NOT_VIEW"), s"unexpected error condition: ${ex.getCondition}") } + + test("rename re-caches a previously cached view") { + // v1 `AlterTableRenameCommand` and v2 `RenameTableExec` both capture the cached storage + // level before rename and re-instate it on the new identifier afterwards. The v2 view + // path (`RenameV2ViewExec`) follows the same pattern -- without it, a user-cached view + // would silently lose its cache entry after RENAME. + val src = s"$catalog.$namespace.v_rename_cached_src" + val dstName = "v_rename_cached_dst" + val dst = s"$catalog.$namespace.$dstName" + createView(src) + spark.catalog.cacheTable(src) + assert(spark.catalog.isCached(src), "bad test: view was not cached in the first place") + try { + sql(s"ALTER VIEW $src RENAME TO $dstName") + // After rename, the destination's plan must still be cached. Resolving the old name + // post-rename throws TABLE_OR_VIEW_NOT_FOUND, so we only check the destination side. + assert(spark.catalog.isCached(dst), + s"$dst should still be cached after RENAME") + } finally { + spark.catalog.uncacheTable(dst) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala index 37cb627468cf..08b10bad7f3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowCreateViewSuiteBase.scala @@ -45,9 +45,11 @@ trait ShowCreateViewSuiteBase extends QueryTest with DDLCommandTestUtils { test("renders the column list") { val view = s"$catalog.$namespace.v_show_create_cols" - sql(s"CREATE VIEW $view (a, b) AS SELECT 1, 2") + sql(s"CREATE VIEW $view (col_alpha, col_beta) AS SELECT 1, 2") val ddl = sql(s"SHOW CREATE TABLE $view").collect().head.getString(0) - assert(ddl.contains("a")) - assert(ddl.contains("b")) + // Use distinctive column names so `contains` checks aren't satisfied by tokens that + // appear elsewhere in the rendered DDL (e.g. inside `CREATE VIEW`, `AS SELECT`, etc.). + assert(ddl.contains("col_alpha"), s"col_alpha missing in DDL: $ddl") + assert(ddl.contains("col_beta"), s"col_beta missing in DDL: $ddl") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala index 53edd698e394..54174387ef3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewRenameSuite.scala @@ -30,4 +30,21 @@ class AlterViewRenameSuite assert(!viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_rename_src"))) assert(viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_rename_dst"))) } + + test("V2: rename to a 2-part target moves the view across namespaces") { + // Exercises `RenameV2ViewExec`'s non-empty-namespace branch -- when the new identifier + // already carries a namespace, the exec passes it through verbatim rather than defaulting + // to the source namespace. + val src = s"$catalog.$namespace.v2_rename_xns_src" + val otherNs = "other_ns" + sql(s"CREATE NAMESPACE IF NOT EXISTS $catalog.$otherNs") + createView(src) + try { + sql(s"ALTER VIEW $src RENAME TO $otherNs.v2_rename_xns_dst") + assert(!viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_rename_xns_src"))) + assert(viewCatalog.viewExists(Identifier.of(Array(otherNs), "v2_rename_xns_dst"))) + } finally { + sql(s"DROP VIEW IF EXISTS $catalog.$otherNs.v2_rename_xns_dst") + } + } } From 09e37e1000ad81ce891243bdd793370f8e3115dc Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 12:17:38 +0000 Subject: [PATCH 07/17] [SPARK-56655][SQL] Self-review fixes: collapse double loadNamespaceMetadata in ApplyDefaultCollation; pin SET TBLPROPERTIES('comment') round-trip through SHOW CREATE TABLE - ApplyDefaultCollation's AlterViewAs case used to call `getCollationFromSchemaMetadata` twice (once in the guard, once in the body), issuing two `loadNamespaceMetadata` round trips per resolution-batch iteration. Restructure to a single lookup and a match-on-Option so the connector sees one round trip. - AlterViewSetTblPropertiesSuiteBase: add a unified test that pins SET TBLPROPERTIES ('comment' = ...) flowing through to SHOW CREATE TABLE on both v1 and v2 paths -- v1 updates the typed `CatalogTable.comment` field, v2 stores via `ViewInfo.properties`, but the user-visible behavior must match. Co-authored-by: Isaac --- .../analysis/ApplyDefaultCollation.scala | 56 ++++++++++--------- .../AlterViewSetTblPropertiesSuiteBase.scala | 15 +++++ 2 files changed, 45 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala index 375a7d3d6e9c..ebc2ac2a6b2f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollation.scala @@ -208,39 +208,43 @@ object ApplyDefaultCollation extends Rule[LogicalPlan] { // We match against ResolvedPersistentView because temporary views don't have a // schema/catalog. The rewrite covers both v1 (session-catalog, [[V1ViewInfo]]) and - // non-session v2 views: when the existing view has no `PROP_COLLATION`, fold the - // namespace's default collation into the resolved `ViewInfo`. For v1, `V1ViewInfo` - // is rebuilt around a `CatalogTable` whose typed `collation` field holds the new - // value; `V1ViewInfo.builderFrom` bridges that into the v2 `properties()` bag, so - // downstream consumers (`fetchDefaultCollation`, `AlterV2ViewExec`'s - // `existingProp(PROP_COLLATION)`) see it under either surface. For v2, we rebuild the - // existing `ViewInfo` with `PROP_COLLATION` set so the same downstream consumers see - // it on the regular `info.properties` path. + // non-session v2 views: when the existing view has no `PROP_COLLATION` and the + // namespace supplies a default, fold that default into the resolved `ViewInfo`. For + // v1, `V1ViewInfo` is rebuilt around a `CatalogTable` whose typed `collation` field + // holds the new value; `V1ViewInfo.builderFrom` bridges that into the v2 + // `properties()` bag, so downstream consumers (`fetchDefaultCollation`, + // `AlterV2ViewExec`'s `existingProp(PROP_COLLATION)`) see it under either surface. + // For v2, we rebuild the existing `ViewInfo` with `PROP_COLLATION` set so the same + // downstream consumers see it on the regular `info.properties` path. case alterViewAs @ AlterViewAs(rpv @ ResolvedPersistentView( catalog: SupportsNamespaces, identifier, info), _, _, _, _) - if Option(info.properties.get(TableCatalog.PROP_COLLATION)).isEmpty && - getCollationFromSchemaMetadata(catalog, identifier.namespace()).nonEmpty => + if Option(info.properties.get(TableCatalog.PROP_COLLATION)).isEmpty => // Only rewrite when the namespace actually supplies a default. [[ViewInfo]] / // [[V1ViewInfo]] are non-case classes, so a copy with structurally-identical fields // still reads as a different reference -- if we rewrote unconditionally, the // resolution batch would see the plan change every iteration and never reach - // fixed point. - val newCollation = - getCollationFromSchemaMetadata(catalog, identifier.namespace()).get - val newInfo: ViewInfo = info match { - case v1Info: V1ViewInfo => - new V1ViewInfo(v1Info.v1Table.copy(collation = Some(newCollation))) - case _ => - CatalogV2Util.viewInfoBuilderFrom(info) - .withCollation(newCollation) - .build() + // fixed point. Looking up the namespace default takes one `loadNamespaceMetadata` + // round trip, so do it once here and bail out before rewriting if the namespace has + // no default. + getCollationFromSchemaMetadata(catalog, identifier.namespace()) match { + case Some(newCollation) => + val newInfo: ViewInfo = info match { + case v1Info: V1ViewInfo => + new V1ViewInfo(v1Info.v1Table.copy(collation = Some(newCollation))) + case _ => + CatalogV2Util.viewInfoBuilderFrom(info) + .withCollation(newCollation) + .build() + } + val newRpv = rpv.copy(info = newInfo) + val newAlterViewAs = CurrentOrigin.withOrigin(alterViewAs.origin) { + alterViewAs.copy(child = newRpv) + } + newAlterViewAs.copyTagsFrom(alterViewAs) + newAlterViewAs + case None => + alterViewAs } - val newRpv = rpv.copy(info = newInfo) - val newAlterViewAs = CurrentOrigin.withOrigin(alterViewAs.origin) { - alterViewAs.copy(child = newRpv) - } - newAlterViewAs.copyTagsFrom(alterViewAs) - newAlterViewAs case createUserDefinedFunction@CreateUserDefinedFunction( ResolvedIdentifier(catalog: SupportsNamespaces, identifier), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala index 68119d7d8e73..b55ee4563c6e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewSetTblPropertiesSuiteBase.scala @@ -96,4 +96,19 @@ trait AlterViewSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestUt val all = sql(s"SHOW TBLPROPERTIES $view").collect() assert(all.length >= 1, s"expected at least one property row, got: ${all.mkString(", ")}") } + + test("setting `comment` flows through to SHOW CREATE TABLE") { + // v1 `AlterTableSetPropertiesCommand` updates the typed `CatalogTable.comment` field when + // the user passes `'comment'` via SET TBLPROPERTIES, so SHOW CREATE TABLE renders the + // comment in the COMMENT clause. The v2 path uses `ViewInfo.properties` as the source of + // truth for `PROP_COMMENT` (see `AlterV2ViewSetPropertiesExec` and `ShowCreateV2ViewExec`), + // so the same SET TBLPROPERTIES('comment' = ...) round-trips through SHOW CREATE TABLE. + // Pin the cross-catalog parity here. + val view = s"$catalog.$namespace.v_set_comment" + createView(view) + sql(s"ALTER VIEW $view SET TBLPROPERTIES ('comment' = 'a view comment')") + val ddl = sql(s"SHOW CREATE TABLE $view").collect().head.getString(0) + assert(ddl.contains("a view comment"), + s"comment did not flow through to SHOW CREATE TABLE: $ddl") + } } From 5173d01e86741efba43188e785ce671720bf5a4f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 13:06:23 +0000 Subject: [PATCH 08/17] [SPARK-56655][SQL][TESTS] Lift v1-parity collision tests for view DDL into Base Both `CREATE VIEW over a non-view table entry` and `DROP VIEW on a non-view table entry` were v2-only tests despite asserting pure SQL behavior. Move the SQL-behavior assertions into the shared *SuiteBase, with a small abstract hook `withSeededTable(qualified)(body)` that each leaf implements via SQL `CREATE TABLE ... USING parquet`. Net effect: v1 also exercises this parity now, and the v2 leaves keep only their genuinely v2-specific post-conditions (catalog-state poking via `getStoredInfo` / `tableExists`). Surfaced one pre-existing v1/v2 divergence in the lift: v1 `DropTableCommand` raises `WRONG_COMMAND_FOR_OBJECT_TYPE` for `DROP VIEW` on a table while v2 `DropViewExec` raises `EXPECT_VIEW_NOT_TABLE`. The lifted Base test accepts either condition with an inline comment; aligning the two error classes is out of scope. Co-authored-by: Isaac --- .../command/CreateViewSuiteBase.scala | 32 ++++++++++++ .../execution/command/DropViewSuiteBase.scala | 25 ++++++++++ .../command/v1/CreateViewSuite.scala | 10 +++- .../execution/command/v1/DropViewSuite.scala | 10 +++- .../command/v2/CreateViewSuite.scala | 49 +++++++------------ .../execution/command/v2/DropViewSuite.scala | 36 +++++++------- 6 files changed, 112 insertions(+), 50 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala index b5c2e584402c..6eaaa7bdd65e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala @@ -38,6 +38,14 @@ trait CreateViewSuiteBase extends QueryTest with DDLCommandTestUtils { } } + /** + * Seed a non-view table at `qualified` (full `catalog.ns.name`) and run `body`. The leaf + * suite implements the path-specific seeding (v1 SQL `CREATE TABLE`, v2 catalog API call) + * and the matching cleanup so the test does not have to know which catalog is under test. + * The seeded table has a single `col STRING` column. + */ + protected def withSeededTable(qualified: String)(body: => Unit): Unit + test("CREATE VIEW persists the body and the SELECT round-trips") { val view = s"$catalog.$namespace.v_create_basic" withSourceTable(1, 2, 3) { @@ -156,4 +164,28 @@ trait CreateViewSuiteBase extends QueryTest with DDLCommandTestUtils { assert(ex.getCondition == "RECURSIVE_VIEW") } } + + test("CREATE VIEW over a non-view table entry surfaces the v1-parity errors") { + // v1-parity error conditions when CREATE [OR REPLACE | IF NOT EXISTS] VIEW collides + // with an existing non-view table at the same identifier. Running on both v1 and v2 + // pins parity from each side -- v1 hits the conditions through the long-established + // session-catalog path, v2 hits them through the new `CreateV2ViewExec`. + val view = s"$catalog.$namespace.v_create_table_collide" + withSeededTable(view) { + // CREATE OR REPLACE VIEW must not silently destroy a non-view table. + val replaceEx = intercept[AnalysisException] { + sql(s"CREATE OR REPLACE VIEW $view AS SELECT 1 AS col") + } + assert(replaceEx.getCondition == "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE") + + // Plain CREATE VIEW over a table surfaces TABLE_OR_VIEW_ALREADY_EXISTS. + val createEx = intercept[AnalysisException] { + sql(s"CREATE VIEW $view AS SELECT 1 AS col") + } + assert(createEx.getCondition == "TABLE_OR_VIEW_ALREADY_EXISTS") + + // CREATE VIEW IF NOT EXISTS is a no-op -- the existing table is left alone. + sql(s"CREATE VIEW IF NOT EXISTS $view AS SELECT 1 AS col") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala index ffa84f32e555..948b572511fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala @@ -35,6 +35,13 @@ trait DropViewSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"SHOW VIEWS IN $catalog.$ns").collect().exists(_.getString(1) == name) } + /** + * Seed a non-view table at `qualified` (full `catalog.ns.name`) and run `body`. The leaf + * suite implements the path-specific seeding (v1 SQL `CREATE TABLE`, v2 catalog API call) + * and the matching cleanup so the test does not have to know which catalog is under test. + */ + protected def withSeededTable(qualified: String)(body: => Unit): Unit + test("drop existing view") { val view = s"$catalog.$namespace.v_drop_basic" sql(s"CREATE VIEW $view AS SELECT 1 AS x") @@ -53,4 +60,22 @@ trait DropViewSuiteBase extends QueryTest with DDLCommandTestUtils { test("drop with IF EXISTS is a no-op when missing") { sql(s"DROP VIEW IF EXISTS $catalog.$namespace.v_drop_never_existed") } + + test("DROP VIEW on a non-view table entry is rejected") { + // Both paths refuse to drop a non-view table when the user said DROP VIEW, but the + // error class differs as a pre-existing divergence: v1 `DropTableCommand` raises + // `WRONG_COMMAND_FOR_OBJECT_TYPE` while v2 `DropViewExec` raises + // `EXPECT_VIEW_NOT_TABLE`. Accept either so this test can run on both paths -- aligning + // the two error classes is out of scope here. + val view = s"$catalog.$namespace.v_drop_table_collide" + withSeededTable(view) { + val ex = intercept[AnalysisException] { + sql(s"DROP VIEW $view") + } + val cond = ex.getCondition + assert( + cond.startsWith("EXPECT_VIEW_NOT_TABLE") || cond == "WRONG_COMMAND_FOR_OBJECT_TYPE", + s"unexpected error condition: $cond") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala index 383fbdb2eca4..13ac3e59767a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala @@ -19,4 +19,12 @@ package org.apache.spark.sql.execution.command.v1 import org.apache.spark.sql.execution.command -class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase +class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase { + + override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { + withTable(qualified) { + sql(s"CREATE TABLE $qualified (col STRING) USING parquet") + body + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala index c8805d67391b..63f83b25e3d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala @@ -19,4 +19,12 @@ package org.apache.spark.sql.execution.command.v1 import org.apache.spark.sql.execution.command -class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase +class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase { + + override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { + withTable(qualified) { + sql(s"CREATE TABLE $qualified (col STRING) USING parquet") + body + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala index 44dee7e6da92..0dc8de1abce6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala @@ -18,13 +18,19 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, TableCatalog, TableInfo, TableSummary} +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, TableCatalog, ViewInfo} import org.apache.spark.sql.execution.command -import org.apache.spark.sql.types.StructType class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase { import testImplicits._ + override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { + withTable(qualified) { + sql(s"CREATE TABLE $qualified (col STRING) USING parquet") + body + } + } + test("V2: CREATE VIEW propagates DEFAULT COLLATION onto the stored ViewInfo") { val view = s"$catalog.$namespace.v2_create_collation" withTable("spark_catalog.default.src_coll") { @@ -53,34 +59,17 @@ class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteB } } - test("V2: CREATE VIEW over a non-view table entry surfaces v1-parity errors") { - val ident = Identifier.of(Array(namespace), "v2_create_table_collide") - val tableInfo = new TableInfo.Builder() - .withSchema(new StructType().add("col", "string")) - .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) - .build() - viewCatalog.createTable(ident, tableInfo) - try { - // CREATE OR REPLACE VIEW must not silently destroy a non-view table -- v1 parity. - val replaceEx = intercept[AnalysisException] { - sql(s"CREATE OR REPLACE VIEW $catalog.$namespace.v2_create_table_collide AS " + - "SELECT 1 AS col") - } - assert(replaceEx.getCondition == "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE") - - // Plain CREATE VIEW over a table surfaces TABLE_OR_VIEW_ALREADY_EXISTS, matching v1. - val createEx = intercept[AnalysisException] { - sql(s"CREATE VIEW $catalog.$namespace.v2_create_table_collide AS SELECT 1 AS col") - } - assert(createEx.getCondition == "TABLE_OR_VIEW_ALREADY_EXISTS") - - // CREATE VIEW IF NOT EXISTS is a no-op -- the table entry is untouched. - sql(s"CREATE VIEW IF NOT EXISTS $catalog.$namespace.v2_create_table_collide AS " + - "SELECT 1 AS col") - val stored = viewCatalog.getStoredInfo(Array(namespace), "v2_create_table_collide") - assert(!stored.isInstanceOf[org.apache.spark.sql.connector.catalog.ViewInfo]) - } finally { - viewCatalog.dropTable(ident) + test("V2: CREATE VIEW IF NOT EXISTS over a table leaves the underlying TableInfo untouched") { + // The Base version of this scenario asserts the SQL behavior (errors / no-op); + // here we additionally pin the v2-only post-condition that the persisted entry under + // the colliding identifier remains a `TableInfo` and is NOT silently swapped for a + // `ViewInfo` by the IF NOT EXISTS path. + val name = "v2_ifne_keeps_table" + val view = s"$catalog.$namespace.$name" + withSeededTable(view) { + sql(s"CREATE VIEW IF NOT EXISTS $view AS SELECT 1 AS col") + val stored = viewCatalog.getStoredInfo(Array(namespace), name) + assert(!stored.isInstanceOf[ViewInfo]) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala index 4ea3dc911c39..ad7bf356ded2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala @@ -18,12 +18,18 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, TableInfo, TableSummary} +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier} import org.apache.spark.sql.execution.command -import org.apache.spark.sql.types.StructType class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase { + override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { + withTable(qualified) { + sql(s"CREATE TABLE $qualified (col STRING) USING parquet") + body + } + } + test("V2: drop removes the entry from the catalog store") { val view = s"$catalog.$namespace.v2_drop_remove" sql(s"CREATE VIEW $view AS SELECT 1 AS x") @@ -32,23 +38,17 @@ class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase assert(!viewCatalog.viewExists(Identifier.of(Array(namespace), "v2_drop_remove"))) } - test("V2: DROP VIEW on a non-view table entry is rejected (v1-parity)") { - val ident = Identifier.of(Array(namespace), "v2_drop_table_collide") - val tableInfo = new TableInfo.Builder() - .withSchema(new StructType().add("x", "int")) - .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) - .build() - viewCatalog.createTable(ident, tableInfo) - try { - val ex = intercept[AnalysisException] { - sql(s"DROP VIEW $catalog.$namespace.v2_drop_table_collide") - } - assert(ex.getCondition.startsWith("EXPECT_VIEW_NOT_TABLE"), - s"expected EXPECT_VIEW_NOT_TABLE, got ${ex.getCondition}") - // The table entry is untouched. + test("V2: DROP VIEW on a non-view table entry leaves the table untouched") { + // The Base version of this scenario asserts the SQL behavior (rejection with + // EXPECT_VIEW_NOT_TABLE); here we additionally pin the v2-only post-condition that + // the underlying entry under the colliding identifier remains a table and was not + // silently dropped by the rejected DROP VIEW. + val name = "v2_drop_keeps_table" + val view = s"$catalog.$namespace.$name" + val ident = Identifier.of(Array(namespace), name) + withSeededTable(view) { + intercept[AnalysisException](sql(s"DROP VIEW $view")) assert(viewCatalog.tableExists(ident)) - } finally { - viewCatalog.dropTable(ident) } } From b6f974ac4aef1b2b68723808f9bad3d281f5c562 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 13:21:07 +0000 Subject: [PATCH 09/17] [SPARK-56655][SQL] Align v2 DROP-wrong-kind error class to v1 (WRONG_COMMAND_FOR_OBJECT_TYPE) The v2 view DDL surface introduced in SPARK-52729 + SPARK-56655 is unreleased, so we can clean up the error class for the two `DROP ` cases without a user-facing break. Changes: - `DropViewExec` (DROP VIEW on a table) and `DropTableExec` (DROP TABLE on a v2 view) switch from `EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE` / `EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE` to `WRONG_COMMAND_FOR_OBJECT_TYPE`. v1's `DropTableCommand` already uses `WRONG_COMMAND_FOR_OBJECT_TYPE` for the same cases, so this aligns v2 to v1 and gives users the actionable hint ("Use DROP TABLE instead." / "Use DROP VIEW instead.") that the `.NO_ALTERNATIVE` subclass omits. Other v2 view DDL "wrong-kind" sites are intentionally left alone: * `EXPECT_*_NOT_*.USE_ALTER_*` already conveys the alternative through its subclass (no information loss to switch). * CREATE OR REPLACE VIEW on a non-view table goes through a helper shared with v1 (released) and has no clean alternative command to suggest. - Tighten the lifted Base test in `DropViewSuiteBase` to assert the now- uniform `WRONG_COMMAND_FOR_OBJECT_TYPE` condition + the rendered "Use DROP TABLE instead" hint, instead of the earlier compromise that accepted either error class. - Inline `withSeededTable` into `CreateViewSuiteBase` / `DropViewSuiteBase`. Both v1 and v2 leaves were implementing it identically (SQL `CREATE TABLE ... USING parquet`) -- `InMemoryTableViewCatalog.createTable` accepts the parquet `TableInfo` the same way the session catalog does -- so the abstract hook was carrying no per-leaf customization. Drop the leaf overrides. Co-authored-by: Isaac --- .../datasources/v2/DropTableExec.scala | 13 +++++--- .../datasources/v2/DropViewExec.scala | 13 +++++--- .../command/CreateViewSuiteBase.scala | 14 +++++--- .../execution/command/DropViewSuiteBase.scala | 32 +++++++++++-------- .../command/v1/CreateViewSuite.scala | 10 +----- .../execution/command/v1/DropViewSuite.scala | 10 +----- .../command/v2/CreateViewSuite.scala | 7 ---- .../execution/command/v2/DropViewSuite.scala | 7 ---- 8 files changed, 47 insertions(+), 59 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala index 18e6a5eb86ac..8d7770d41461 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.quoteIfNeeded import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, ViewCatalog} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.util.ArrayImplicits._ @@ -33,8 +34,8 @@ import org.apache.spark.util.ArrayImplicits._ * * When the table is absent, falls back to `viewExists` for catalogs that also implement * [[ViewCatalog]] -- distinguishes "wrong type" from "missing" so a `DROP TABLE someView` - * on a mixed catalog surfaces the dedicated `EXPECT_TABLE_NOT_VIEW` error rather than a - * generic "table not found", matching the v1 `DropTableCommand(isView = false)` behavior. + * on a mixed catalog surfaces `WRONG_COMMAND_FOR_OBJECT_TYPE` ("Use DROP VIEW instead") + * rather than a generic "table not found", matching v1 `DropTableCommand(isView = false)`. */ case class DropTableExec( catalog: TableCatalog, @@ -52,8 +53,12 @@ case class DropTableExec( (catalog.name() +: ident.namespace() :+ ident.name()).toImmutableArraySeq catalog match { case vc: ViewCatalog if vc.viewExists(ident) => - throw QueryCompilationErrors.expectTableNotViewError( - nameParts, cmd = "DROP TABLE", suggestAlternative = false, t = this) + throw QueryCompilationErrors.wrongCommandForObjectTypeError( + operation = "DROP TABLE", + requiredType = "TABLE", + objectName = nameParts.map(quoteIfNeeded).mkString("."), + foundType = "VIEW", + alternative = "DROP VIEW") case _ if !ifExists => throw QueryCompilationErrors.noSuchTableError(nameParts) case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropViewExec.scala index 9a665f644e0d..c1655402d368 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropViewExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NoSuchViewException import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.quoteIfNeeded import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, ViewCatalog} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.util.ArrayImplicits._ @@ -27,8 +28,8 @@ import org.apache.spark.util.ArrayImplicits._ /** * Physical plan node for DROP VIEW on a v2 [[ViewCatalog]]. Calls [[ViewCatalog#dropView]]; if * it returns false and the catalog also implements [[TableCatalog]] with a table at this - * identifier, surfaces the dedicated `EXPECT_VIEW_NOT_TABLE` error rather than a generic - * "view not found" -- matching v1 `DropTableCommand(isView = true)`. + * identifier, surfaces `WRONG_COMMAND_FOR_OBJECT_TYPE` ("Use DROP TABLE instead") rather than + * a generic "view not found" -- matching v1 `DropTableCommand(isView = true)`. */ case class DropViewExec( catalog: ViewCatalog, @@ -45,8 +46,12 @@ case class DropViewExec( (catalog.name() +: ident.namespace() :+ ident.name()).toImmutableArraySeq catalog match { case tc: TableCatalog if tc.tableExists(ident) => - throw QueryCompilationErrors.expectViewNotTableError( - nameParts, cmd = "DROP VIEW", suggestAlternative = false, t = this) + throw QueryCompilationErrors.wrongCommandForObjectTypeError( + operation = "DROP VIEW", + requiredType = "VIEW", + objectName = nameParts.map(quoteIfNeeded).mkString("."), + foundType = "TABLE", + alternative = "DROP TABLE") case _ if !ifExists => throw new NoSuchViewException(ident) case _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala index 6eaaa7bdd65e..d046e74b6862 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateViewSuiteBase.scala @@ -39,12 +39,16 @@ trait CreateViewSuiteBase extends QueryTest with DDLCommandTestUtils { } /** - * Seed a non-view table at `qualified` (full `catalog.ns.name`) and run `body`. The leaf - * suite implements the path-specific seeding (v1 SQL `CREATE TABLE`, v2 catalog API call) - * and the matching cleanup so the test does not have to know which catalog is under test. - * The seeded table has a single `col STRING` column. + * Seed a non-view table at `qualified` (full `catalog.ns.name`) and run `body`. Same SQL + * for v1 and v2 -- `InMemoryTableViewCatalog.createTable` accepts the parquet TableInfo + * the same way the session catalog does, so both legs share this implementation. */ - protected def withSeededTable(qualified: String)(body: => Unit): Unit + protected final def withSeededTable(qualified: String)(body: => Unit): Unit = { + withTable(qualified) { + sql(s"CREATE TABLE $qualified (col STRING) USING parquet") + body + } + } test("CREATE VIEW persists the body and the SELECT round-trips") { val view = s"$catalog.$namespace.v_create_basic" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala index 948b572511fb..92a368e4155a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropViewSuiteBase.scala @@ -36,11 +36,16 @@ trait DropViewSuiteBase extends QueryTest with DDLCommandTestUtils { } /** - * Seed a non-view table at `qualified` (full `catalog.ns.name`) and run `body`. The leaf - * suite implements the path-specific seeding (v1 SQL `CREATE TABLE`, v2 catalog API call) - * and the matching cleanup so the test does not have to know which catalog is under test. + * Seed a non-view table at `qualified` (full `catalog.ns.name`) and run `body`. Same SQL + * for v1 and v2 -- `InMemoryTableViewCatalog.createTable` accepts the parquet TableInfo + * the same way the session catalog does, so both legs share this implementation. */ - protected def withSeededTable(qualified: String)(body: => Unit): Unit + protected final def withSeededTable(qualified: String)(body: => Unit): Unit = { + withTable(qualified) { + sql(s"CREATE TABLE $qualified (col STRING) USING parquet") + body + } + } test("drop existing view") { val view = s"$catalog.$namespace.v_drop_basic" @@ -61,21 +66,20 @@ trait DropViewSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"DROP VIEW IF EXISTS $catalog.$namespace.v_drop_never_existed") } - test("DROP VIEW on a non-view table entry is rejected") { - // Both paths refuse to drop a non-view table when the user said DROP VIEW, but the - // error class differs as a pre-existing divergence: v1 `DropTableCommand` raises - // `WRONG_COMMAND_FOR_OBJECT_TYPE` while v2 `DropViewExec` raises - // `EXPECT_VIEW_NOT_TABLE`. Accept either so this test can run on both paths -- aligning - // the two error classes is out of scope here. + test("DROP VIEW on a non-view table entry surfaces WRONG_COMMAND_FOR_OBJECT_TYPE") { + // Both v1 `DropTableCommand` and v2 `DropViewExec` route this case to + // `WRONG_COMMAND_FOR_OBJECT_TYPE`, which renders "Use DROP TABLE instead" -- giving the + // user the right command to retry. The `alternative` parameter on the rendered message + // surfaces the suggestion that subclassed `EXPECT_*` errors otherwise carry only via + // their subclass name. val view = s"$catalog.$namespace.v_drop_table_collide" withSeededTable(view) { val ex = intercept[AnalysisException] { sql(s"DROP VIEW $view") } - val cond = ex.getCondition - assert( - cond.startsWith("EXPECT_VIEW_NOT_TABLE") || cond == "WRONG_COMMAND_FOR_OBJECT_TYPE", - s"unexpected error condition: $cond") + assert(ex.getCondition == "WRONG_COMMAND_FOR_OBJECT_TYPE", + s"unexpected error condition: ${ex.getCondition}") + assert(ex.getMessage.contains("Use DROP TABLE instead")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala index 13ac3e59767a..383fbdb2eca4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateViewSuite.scala @@ -19,12 +19,4 @@ package org.apache.spark.sql.execution.command.v1 import org.apache.spark.sql.execution.command -class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase { - - override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { - withTable(qualified) { - sql(s"CREATE TABLE $qualified (col STRING) USING parquet") - body - } - } -} +class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala index 63f83b25e3d1..c8805d67391b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropViewSuite.scala @@ -19,12 +19,4 @@ package org.apache.spark.sql.execution.command.v1 import org.apache.spark.sql.execution.command -class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase { - - override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { - withTable(qualified) { - sql(s"CREATE TABLE $qualified (col STRING) USING parquet") - body - } - } -} +class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala index 0dc8de1abce6..1750a538b1d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateViewSuite.scala @@ -24,13 +24,6 @@ import org.apache.spark.sql.execution.command class CreateViewSuite extends command.CreateViewSuiteBase with ViewCommandSuiteBase { import testImplicits._ - override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { - withTable(qualified) { - sql(s"CREATE TABLE $qualified (col STRING) USING parquet") - body - } - } - test("V2: CREATE VIEW propagates DEFAULT COLLATION onto the stored ViewInfo") { val view = s"$catalog.$namespace.v2_create_collation" withTable("spark_catalog.default.src_coll") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala index ad7bf356ded2..cf8ec930fddb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropViewSuite.scala @@ -23,13 +23,6 @@ import org.apache.spark.sql.execution.command class DropViewSuite extends command.DropViewSuiteBase with ViewCommandSuiteBase { - override protected def withSeededTable(qualified: String)(body: => Unit): Unit = { - withTable(qualified) { - sql(s"CREATE TABLE $qualified (col STRING) USING parquet") - body - } - } - test("V2: drop removes the entry from the catalog store") { val view = s"$catalog.$namespace.v2_drop_remove" sql(s"CREATE VIEW $view AS SELECT 1 AS x") From 84260d65389a1115568b013d00ff796ccf4e290c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 13:48:36 +0000 Subject: [PATCH 10/17] [SPARK-56655][SQL] Fix CREATE TABLE LIKE on v2 view source via DataSourceV2Strategy Restore the v2 strategy match for CreateTableLike when the source is a non-session v2 view: previously the case narrowed to V1ViewInfo only, so a v2 ViewInfo source fell through to a runtime MatchError. Synthesize the CatalogTable via V1Table.toCatalogTable for non-V1ViewInfo views, mirroring the pre-narrowing behavior. Co-authored-by: Isaac --- .../execution/datasources/v2/DataSourceV2Strategy.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index f51f9159ecf0..c7a0d6515bae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -265,13 +265,18 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // CREATE TABLE ... LIKE ... for a v2 catalog target. // Source is an already-resolved Table object; no extra catalog round-trip is needed. - // Views are wrapped in V1Table so the exec can extract schema and provider uniformly. + // Views are wrapped in V1Table so the exec can extract schema and provider uniformly -- + // session-catalog (v1) views unwrap to their original `CatalogTable`; non-session v2 + // views go through `V1Table.toCatalogTable` to synthesize an equivalent `CatalogTable` + // from the resolved `ViewInfo`. case CreateTableLike( ResolvedIdentifier(catalog, ident), source, locationStr, provider, serdeInfo, properties, ifNotExists) => val table = source match { case ResolvedTable(_, _, t, _) => t case ResolvedPersistentView(_, _, info: V1ViewInfo) => V1Table(info.v1Table) + case rpv @ ResolvedPersistentView(viewCatalog, viewIdent, _) => + V1Table(V1Table.toCatalogTable(viewCatalog, viewIdent, rpv.info)) case ResolvedTempView(_, meta) => V1Table(meta) } val location = locationStr.map { loc => From cf0abfa69c31b4e36ef807efe9f58f333460c089 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 14:21:33 +0000 Subject: [PATCH 11/17] [SPARK-56655][SQL] Self-review fixes: pin DESC TABLE PARTITION on v2 view; restore DESC AS JSON for non-session v2 views - ResolveSessionCatalog: rewrite `DESCRIBE TABLE PARTITION (...)` on a non-session v2 view (where the existing v1 rewrite is gated on `ResolvedV1TableOrViewIdentifier`) to throw `descPartitionNotAllowedOnView` early. Without this, `UnresolvedPartitionSpec` is never resolved on the v2 view path and CheckAnalysis surfaces a generic "Found the unresolved operator" INTERNAL_ERROR. Mirrors the v1 runtime check in `DescribeTableCommand.describeDetailedPartitionInfo`. - DescribeRelationJsonCommand: restore pre-PR behavior of `DESC ... AS JSON` on non-session v2 views. The first cut narrowed the view branch to `V1ViewInfo` only, so non-session v2 views fell through to `describeAsJsonNotSupportedForV2TablesError` -- a regression vs. master (where the synthesized `CatalogTable` made the JSON path uniform). Restore the uniform behavior by recovering the v1 `CatalogTable` from `V1ViewInfo` and synthesizing an equivalent via `V1Table.toCatalogTable` for v2 views, mirroring the `CreateTableLike` strategy case. - New tests in `DataSourceV2MetadataViewSuite` for both fixes. Co-authored-by: Isaac --- .../analysis/ResolveSessionCatalog.scala | 11 ++++++++ .../command/DescribeRelationJsonCommand.scala | 16 +++++++---- .../DataSourceV2MetadataViewSuite.scala | 28 +++++++++++++++++++ 3 files changed, 49 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 2f03498d4ce2..07731d6db836 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -208,6 +208,17 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) output) => DescribeTableCommand(resolvedChild, ident, spec, isExtended, output) + // `DESCRIBE TABLE PARTITION (...)` against a non-session v2 view: the v1 rewrite + // above is gated on `ResolvedV1TableOrViewIdentifier` (session-only), so non-session v2 + // views fall through. Reject early with the same `FORBIDDEN_OPERATION` v1 raises at + // runtime in `DescribeTableCommand.describeDetailedPartitionInfo`. Without this rewrite, + // CheckAnalysis surfaces a generic "Found the unresolved operator" INTERNAL_ERROR + // because `UnresolvedPartitionSpec` is never resolved on the v2 view path. + case DescribeTablePartition(rpv: ResolvedPersistentView, _, _, _) => + val quoted = (rpv.catalog.name() +: rpv.identifier.asMultipartIdentifier) + .map(quoteIfNeeded).mkString(".") + throw QueryCompilationErrors.descPartitionNotAllowedOnView(quoted) + case DescribeColumn(ResolvedViewIdentifier(ident), column, isExtended, output) => // `ResolvedPersistentView` exposes the view's schema as its `output`, so `ResolveReferences` // typically resolves the column to an `Attribute` here. We also accept the legacy diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala index 562aab876e70..86d13a0bdd23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala @@ -67,15 +67,19 @@ case class DescribeRelationJsonCommand( describeColsJson(v.metadata.schema, jsonMap) describeFormattedTableInfoJson(v.metadata, jsonMap) - case v @ ResolvedPersistentView(_, _, v1Info: V1ViewInfo) => - // Only session-catalog (v1) views are supported here. Non-session v2 views fall - // through to the catch-all below and surface - // `describeAsJsonNotSupportedForV2TablesError`, mirroring the v2 table path -- there - // is no `DESC ... AS JSON` exec on the v2 view side yet. + case v: ResolvedPersistentView => if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(v.identifier.name()) } - val metadata = v1Info.v1Table + // For session-catalog (v1) views, recover the original `CatalogTable` from + // `V1ViewInfo`. For non-session v2 views (which carry a plain `ViewInfo`), synthesize + // an equivalent `CatalogTable` via `V1Table.toCatalogTable` -- mirrors the + // `CreateTableLike` strategy case in `DataSourceV2Strategy` and preserves pre-PR + // behavior of `DESC ... AS JSON` working on any persistent view. + val metadata = v.info match { + case v1Info: V1ViewInfo => v1Info.v1Table + case info => V1Table.toCatalogTable(v.catalog, v.identifier, info) + } describeIdentifier(v.identifier.toQualifiedNameParts(v.catalog), jsonMap) describeColsJson(metadata.schema, jsonMap) describeFormattedTableInfoJson(metadata, jsonMap) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala index 25879deb02c1..0921e9e74781 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala @@ -325,6 +325,34 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { "ANALYZE TABLE view_catalog.default.v_analyze_cols COMPUTE STATISTICS FOR COLUMNS x") } + test("DESCRIBE TABLE ... PARTITION on a v2 view is rejected") { + // The parser builds an `UnresolvedTableOrView` for DESCRIBE, so this reaches the v2 + // strategy with a `ResolvedPersistentView` child. Without an explicit pin the planner + // falls through to a "No plan for DescribeTablePartition" assertion; pin it with + // FORBIDDEN_OPERATION/DESC PARTITION on VIEW to mirror the v1 runtime check in + // `DescribeTableCommand.describeDetailedPartitionInfo`. + seedV2View("v_desc_part") + val ex = intercept[AnalysisException] { + sql("DESCRIBE TABLE view_catalog.default.v_desc_part PARTITION (x = 1)") + } + assert(ex.getCondition == "FORBIDDEN_OPERATION", s"got ${ex.getCondition}") + } + + test("DESCRIBE TABLE EXTENDED ... AS JSON on a v2 view succeeds") { + // Pre-PR, `ResolvedPersistentView.metadata: CatalogTable` was always populated (a + // synthesized `CatalogTable` for v2 views), so DESC ... AS JSON worked uniformly. Post + // ResolvedPersistentView -> `info: ViewInfo`, the JSON command now synthesizes a + // `CatalogTable` via `V1Table.toCatalogTable` for non-session v2 views to preserve that + // pre-PR behavior. + seedV2View("v_desc_json") + val rows = sql( + "DESCRIBE TABLE EXTENDED view_catalog.default.v_desc_json AS JSON").collect() + assert(rows.length == 1, s"DESC AS JSON should produce one row, got: ${rows.length}") + val json = rows.head.getString(0) + assert(json.contains("\"v_desc_json\""), s"JSON output missing view name: $json") + assert(json.contains("\"VIEW\""), s"JSON output missing VIEW table_type: $json") + } + // DROP VIEW behavior tests live in the per-catalog triplet // `sql.execution.command.{,v1/,v2/}.DropViewSuite{,Base}`. From 87847874a24bd2101d6edb61ba9149d2e5e0e6a1 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 14:55:58 +0000 Subject: [PATCH 12/17] [SPARK-56655][SQL] Self-review fixes: SHOW COLUMNS namespace conflict; DESCRIBE EXTENDED first-class fields; shared DescribeColumn helper; clear queryColumnNames on EVOLUTION Four self-review findings rolled into one commit: 1. SHOW COLUMNS namespace conflict on v2 view (Pass A finding 1). The v2 strategy case `case ShowColumns(rpv @ ResolvedPersistentView(_, _, _), _, output)` discarded the explicit `FROM ` argument; v1 validates that the view's resolved namespace matches and errors with `SHOW_COLUMNS_WITH_CONFLICT_NAMESPACE`. Add the same cross-check to the v2 case (multi-part-namespace-aware via `Identifier.namespace().toSeq`) and a covering test in `ShowViewColumnsSuiteBase` so both v1 and v2 legs assert the same error condition. 2. DESCRIBE TABLE EXTENDED on v2 view promotes first-class fields (Pass A finding 2). v1 `CatalogTable.toJsonLinkedHashMap` renders Owner / Comment / Collation as their own rows above the generic Table Properties block; the first cut of `DescribeV2ViewExec` collapsed all three into the Properties string. Promote them to dedicated rows (matching the v1 row order: Owner first, then Comment, then Collation, before the View Text block) so users on a v2 catalog see the same shape as on the session catalog. New test in `DescribeViewSuiteBase` covers Comment + Collation on both legs. 3. Shared DescribeColumn name-parts helper (Pass A finding 3). The `column match { UnresolvedAttribute / Attribute / Alias / _ }` block was duplicated verbatim in `ResolveSessionCatalog` (v1 view rewrite) and `DataSourceV2Strategy` (v2 view case) -- both paths added by this PR. Extract to `DescribeColumn.extractColumnNameParts` on the companion object in `v2Commands.scala`; both call sites now go through it. Clean up the now- unused `Alias` and `UnresolvedAttribute` imports in the strategy file. 4. Clear queryColumnNames when ALTER VIEW switches to EVOLUTION (Pass B finding 1). v1 `generateViewProperties` empties the view-query-output column list when `viewSchemaMode == SchemaEvolution` because EVOLUTION always uses the view's current schema as the column source; the new `AlterV2ViewSchemaBindingExec` was preserving the existing array via `viewInfoBuilderFrom(existing).withSchemaMode(...)`, leaving non-canonical metadata in the catalog. Add the explicit `withQueryColumnNames(empty)` for the EVOLUTION branch and a test in `v2.AlterViewSchemaBindingSuite` that pins the cleared-then-restored shape. Co-authored-by: Isaac --- .../catalyst/plans/logical/v2Commands.scala | 30 ++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 11 ++---- .../datasources/v2/AlterV2ViewExec.scala | 14 +++++--- .../datasources/v2/DataSourceV2Strategy.scala | 35 +++++++++++-------- .../v2/V2ViewInspectionExecs.scala | 12 +++++++ .../command/DescribeViewSuiteBase.scala | 17 +++++++++ .../command/ShowViewColumnsSuiteBase.scala | 15 +++++++- .../v2/AlterViewSchemaBindingSuite.scala | 26 ++++++++++++++ 8 files changed, 131 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 0eded2d9dbdf..aef043178a29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -949,6 +949,36 @@ case class DescribeColumn( object DescribeColumn { def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes() + + /** + * Extract the column nameParts from the (possibly resolved) column expression on a + * `DescribeColumn` command. Used by both the v1 rewrite in `ResolveSessionCatalog` and the + * v2 strategy case in `DataSourceV2Strategy` -- centralizing the unwrap means the two paths + * cannot drift. + * + * `ResolveReferences` typically resolves the column against the relation's `output`, so we + * see an `Attribute` here. The legacy `UnresolvedAttribute` form is also accepted (e.g. when + * the column name doesn't exist in the relation and resolution is skipped). `Alias` + * indicates a nested-column reference (`a.b`) which `ResolveReferences` rewrites to + * `Alias(GetStructField(...), b)` -- nested columns are unsupported on this command. + */ + def extractColumnNameParts(column: org.apache.spark.sql.catalyst.expressions.Expression) + : Seq[String] = { + import org.apache.spark.SparkException + import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute + import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} + import org.apache.spark.sql.catalyst.util.toPrettySQL + import org.apache.spark.sql.errors.QueryCompilationErrors + column match { + case u: UnresolvedAttribute => u.nameParts + case a: Attribute => a.qualifier :+ a.name + case Alias(child, _) => + throw QueryCompilationErrors.commandNotSupportNestedColumnError( + "DESC TABLE COLUMN", toPrettySQL(child)) + case _ => + throw SparkException.internalError(s"[BUG] unexpected column expression: $column") + } + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 07731d6db836..29bcb98f3f09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -224,15 +224,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // typically resolves the column to an `Attribute` here. We also accept the legacy // `UnresolvedAttribute` form (e.g. the parser referenced a non-existent column whose // resolution was skipped) so the rewrite stays robust across analyzer ordering changes. - val nameParts = column match { - case u: UnresolvedAttribute => u.nameParts - case a: Attribute => a.qualifier :+ a.name - case Alias(child, _) => - throw QueryCompilationErrors.commandNotSupportNestedColumnError( - "DESC TABLE COLUMN", toPrettySQL(child)) - case _ => - throw SparkException.internalError(s"[BUG] unexpected column expression: $column") - } + // The unwrap logic is shared with the non-session v2 view path in `DataSourceV2Strategy`. + val nameParts = DescribeColumn.extractColumnNameParts(column) DescribeColumnCommand(ident, nameParts, isExtended, output) case DescribeColumn(ResolvedV1TableIdentifier(ident), column, isExtended, output) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala index 417234a01733..2309cb31b5eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewExec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ViewSchemaMode} +import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, SchemaEvolution, ViewSchemaMode} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog, ViewCatalog, ViewInfo} @@ -144,7 +144,10 @@ case class AlterV2ViewUnsetPropertiesExec( * Physical plan node for ALTER VIEW ... WITH SCHEMA BINDING on a v2 [[ViewCatalog]]. Replaces * the schema-binding mode on the analysis-time view payload and dispatches to * [[ViewCatalog#replaceView]]. The view body itself is not re-analyzed -- only the binding mode - * field changes. + * field changes. When the new mode is EVOLUTION, `queryColumnNames` is also cleared, mirroring + * v1 `generateViewProperties` -- in EVOLUTION mode the view always uses its current schema as + * the column source, so leaving stale `queryColumnNames` would produce non-canonical persisted + * metadata. */ case class AlterV2ViewSchemaBindingExec( catalog: ViewCatalog, @@ -155,9 +158,12 @@ case class AlterV2ViewSchemaBindingExec( override def output: Seq[org.apache.spark.sql.catalyst.expressions.Attribute] = Seq.empty override protected def run(): Seq[InternalRow] = { - val info = CatalogV2Util.viewInfoBuilderFrom(existingView) + val builder = CatalogV2Util.viewInfoBuilderFrom(existingView) .withSchemaMode(viewSchemaMode.toString) - .build() + if (viewSchemaMode == SchemaEvolution) { + builder.withQueryColumnNames(Array.empty[String]) + } + val info = builder.build() CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) catalog.replaceView(identifier, info) Seq.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index c7a0d6515bae..243ab10f2a6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -24,10 +24,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkIllegalArgumentException} import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EXPR -import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedPersistentView, ResolvedTable, ResolvedTempView, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedPersistentView, ResolvedTable, ResolvedTempView} import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, DynamicPruning, Expression, NamedExpression, Not, Or, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning, Expression, NamedExpression, Not, Or, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ @@ -363,7 +363,20 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val quoted = (catalog.name() +: ident.asMultipartIdentifier).map(quoteIfNeeded).mkString(".") ShowV2ViewPropertiesExec(output, quoted, rpv.info, propertyKey) :: Nil - case ShowColumns(rpv @ ResolvedPersistentView(_, _, _), _, output) => + case ShowColumns(rpv @ ResolvedPersistentView(_, ident, _), ns, output) => + // If `SHOW COLUMNS IN FROM ` was written with both the view's namespace and + // an explicit `FROM `, validate they agree -- mirrors the v1 rewrite in + // `ResolveSessionCatalog`. For multi-level v2 namespaces we compare the full namespace + // sequence (case-insensitively) rather than v1's single-part `database` check. + ns.foreach { nsSeq => + val resolver = session.sessionState.conf.resolver + val viewNs = ident.namespace().toSeq + val mismatch = viewNs.length != nsSeq.length || + viewNs.zip(nsSeq).exists { case (a, b) => !resolver(a, b) } + if (mismatch) { + throw QueryCompilationErrors.showColumnsWithConflictNamespacesError(nsSeq, viewNs) + } + } ShowV2ViewColumnsExec(output, rpv.info) :: Nil case DescribeRelation(rpv @ ResolvedPersistentView(catalog, ident, _), isExtended, output) => @@ -372,18 +385,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DescribeColumn(rpv @ ResolvedPersistentView(_, _, _), column, isExtended, output) => // `ResolvedPersistentView.output` exposes the view's schema, so `ResolveReferences` // resolves the column against it -- meaning we typically receive an `Attribute` here. - // Accept the legacy `UnresolvedAttribute` form too, mirroring the v1 rewrite for - // session-catalog views in `ResolveSessionCatalog`. - val nameParts = column match { - case u: UnresolvedAttribute => u.nameParts - case a: Attribute => a.qualifier :+ a.name - case Alias(child, _) => - throw QueryCompilationErrors.commandNotSupportNestedColumnError( - "DESC TABLE COLUMN", toPrettySQL(child)) - case _ => - throw SparkException.internalError(s"[BUG] unexpected column expression: $column") - } - DescribeV2ViewColumnExec(output, rpv.info, nameParts, isExtended) :: Nil + // Accept the legacy `UnresolvedAttribute` form too. The unwrap logic is shared with the + // v1 rewrite for session-catalog views in `ResolveSessionCatalog`. + DescribeV2ViewColumnExec( + output, rpv.info, DescribeColumn.extractColumnNameParts(column), isExtended) :: Nil // Plans that resolve through `UnresolvedTableOrView` reach here with a // `ResolvedPersistentView` child for non-session v2 views (the v1 rewrite in diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala index ad5659c6bc8c..5c9fbbb7eb0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ViewInspectionExecs.scala @@ -170,6 +170,18 @@ case class DescribeV2ViewExec( val qualified = (identifier.namespace() :+ identifier.name()) .map(quoteIfNeeded).mkString(".") result += toCatalystRow("Identifier", qualified, "") + // Promote first-class reserved fields (Owner / Comment / Collation) to top-level rows + // before the EXTENDED Properties block, mirroring v1 `CatalogTable.toJsonLinkedHashMap` + // which renders these as their own rows rather than burying them in `Table Properties`. + Option(viewInfo.properties.get(TableCatalog.PROP_OWNER)).filter(_.nonEmpty).foreach { o => + result += toCatalystRow("Owner", o, "") + } + Option(viewInfo.properties.get(TableCatalog.PROP_COMMENT)).foreach { c => + result += toCatalystRow("Comment", c, "") + } + Option(viewInfo.properties.get(TableCatalog.PROP_COLLATION)).foreach { c => + result += toCatalystRow("Collation", c, "") + } result += toCatalystRow("View Text", viewInfo.queryText, "") Option(viewInfo.currentCatalog).foreach { c => result += toCatalystRow("View Current Catalog", c, "") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala index ff745a9b8028..9f3b9a516426 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala @@ -47,4 +47,21 @@ trait DescribeViewSuiteBase extends QueryTest with DDLCommandTestUtils { rows.contains("# Detailed View Information"), s"expected a detailed-info block in:\n${rows.mkString("\n")}") } + + test("describe extended promotes Comment and Collation to top-level rows") { + // v1 `CatalogTable.toJsonLinkedHashMap` and v2 `DescribeV2ViewExec` both render Comment / + // Collation as their own rows in the EXTENDED block, separately from the generic + // Properties row, so users don't have to scrape the Properties string for first-class + // fields. + val view = s"$catalog.$namespace.v_describe_first_class" + sql(s"CREATE VIEW $view COMMENT 'hello' DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS x") + val rows = sql(s"DESCRIBE TABLE EXTENDED $view").collect().map { r => + r.getString(0) -> Option(r.getString(1)).getOrElse("") + }.toMap + assert(rows.get("Comment").contains("hello"), + s"expected Comment=hello in:\n$rows") + // v1 renders the collation name verbatim (UTF8_LCASE); v2 does the same. + assert(rows.get("Collation").exists(_.equalsIgnoreCase("UTF8_LCASE")), + s"expected Collation=UTF8_LCASE in:\n$rows") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala index 7273db1587d7..99997118b8e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewColumnsSuiteBase.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{AnalysisException, QueryTest} /** * Unified tests for `SHOW COLUMNS` against a view, on V1 (session) and V2 view catalogs. @@ -40,4 +40,17 @@ trait ShowViewColumnsSuiteBase extends QueryTest with DDLCommandTestUtils { val cols = sql(s"SHOW COLUMNS IN $view").collect().map(_.getString(0)).toSeq assert(cols == Seq("alpha", "beta")) } + + test("FROM mismatching the view's namespace is rejected") { + // `SHOW COLUMNS IN FROM ` cross-checks the view's resolved namespace against + // the explicit FROM namespace. Mismatch must error rather than silently ignoring FROM -- + // v1 enforces this in `ResolveSessionCatalog`; v2 enforces it in `DataSourceV2Strategy`. + val view = s"$catalog.$namespace.v_show_cols_ns" + sql(s"CREATE VIEW $view AS SELECT 1 AS a") + val ex = intercept[AnalysisException] { + sql(s"SHOW COLUMNS IN $view FROM other_ns") + } + assert(ex.getCondition == "SHOW_COLUMNS_WITH_CONFLICT_NAMESPACE", + s"unexpected error condition: ${ex.getCondition}") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala index 14112a6e0c87..517880047d25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterViewSchemaBindingSuite.scala @@ -29,4 +29,30 @@ class AlterViewSchemaBindingSuite val stored = viewCatalog.getStoredView(Array(namespace), "v2_schema_mode") assert(stored.schemaMode == "EVOLUTION") } + + test("V2: switching to EVOLUTION clears queryColumnNames; switching back restores them") { + // Mirrors v1 `generateViewProperties`: in EVOLUTION mode the view always uses its current + // schema as the column source, so persisting `queryColumnNames` would be non-canonical. + // After flipping back to BINDING via a CREATE OR REPLACE (which re-captures column names), + // the field must be populated again so view-text expansion has the original aliases. + val name = "v2_schema_mode_qcols" + val view = s"$catalog.$namespace.$name" + sql(s"CREATE VIEW $view AS SELECT 1 AS x, 2 AS y") + val initial = viewCatalog.getStoredView(Array(namespace), name) + assert(initial.queryColumnNames.toSeq == Seq("x", "y")) + + sql(s"ALTER VIEW $view WITH SCHEMA EVOLUTION") + val afterEvo = viewCatalog.getStoredView(Array(namespace), name) + assert(afterEvo.schemaMode == "EVOLUTION") + assert(afterEvo.queryColumnNames.isEmpty, + "queryColumnNames must be cleared in EVOLUTION mode") + + sql(s"ALTER VIEW $view WITH SCHEMA BINDING") + val afterBinding = viewCatalog.getStoredView(Array(namespace), name) + assert(afterBinding.schemaMode == "BINDING") + // ALTER VIEW WITH SCHEMA BINDING does not re-analyze the view body; the queryColumnNames + // field stays at whatever ALTER VIEW WITH SCHEMA EVOLUTION left it as. Users who want the + // original aliases back run CREATE OR REPLACE VIEW, which re-captures them. + assert(afterBinding.queryColumnNames.isEmpty) + } } From dabfd4f3f9b52f192afb40d404218c9d049f5df6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 15:28:24 +0000 Subject: [PATCH 13/17] [SPARK-56655][SQL][TESTS] Lift two v2 view-DDL SQL-behavior tests into Base Two v2-only tests in the per-command triplets were asserting pure SQL behavior that v1 hits identically. Move them into the shared *SuiteBase so both v1 and v2 catalogs exercise the same assertions, leaving each v2 leaf with only genuinely v2-specific tests (catalog state poking, v2-only error scenarios, v2-native output formats): - `ShowViewsSuiteBase` gains "does not include non-view table entries". v1 routes `SHOW VIEWS` through the session catalog with a views-only filter; v2 routes it through `ViewCatalog.listViews`. Both must exclude tables that share the same namespace and mark `isTemporary` as false. The seed table is created via SQL `CREATE TABLE ... USING parquet`, accepted uniformly by the session catalog and `InMemoryTableViewCatalog`. - `DescribeViewSuiteBase` gains "describe extended includes Catalog and View Text rows". Both v1 `DescribeTableCommand` and v2 `DescribeV2ViewExec` emit these rows in the EXTENDED block; pinning them in the Base catches any future regression on either path. The v2 leaf retains only the `# Detailed View Information` header test -- v1 emits `# Detailed Table Information` for views, so this assertion is genuinely v2-specific and shouldn't be lifted. Co-authored-by: Isaac --- .../command/DescribeViewSuiteBase.scala | 15 +++++++++++ .../command/ShowViewsSuiteBase.scala | 20 +++++++++++++++ .../command/v2/DescribeViewSuite.scala | 13 +++------- .../execution/command/v2/ShowViewsSuite.scala | 25 +------------------ 4 files changed, 40 insertions(+), 33 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala index 9f3b9a516426..4520a9580e02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeViewSuiteBase.scala @@ -48,6 +48,21 @@ trait DescribeViewSuiteBase extends QueryTest with DDLCommandTestUtils { s"expected a detailed-info block in:\n${rows.mkString("\n")}") } + test("describe extended includes Catalog and View Text rows") { + // Both v1 (`DescribeTableCommand` over a `CatalogTable` of type VIEW) and v2 + // (`DescribeV2ViewExec`) emit a `Catalog` row carrying the resolved catalog name and a + // `View Text` row containing the view body, so users can read the actual definition out + // of EXTENDED rather than going to SHOW CREATE TABLE for it. + val view = s"$catalog.$namespace.v_describe_ext_body" + sql(s"CREATE VIEW $view AS SELECT 7 AS x") + val rows = sql(s"DESCRIBE TABLE EXTENDED $view").collect() + val pairs = rows.map(r => r.getString(0) -> Option(r.getString(1)).getOrElse("")).toMap + assert(pairs.get("Catalog").contains(catalog), + s"expected Catalog=$catalog in:\n$pairs") + assert(pairs.get("View Text").exists(_.contains("SELECT 7 AS x")), + s"expected View Text containing 'SELECT 7 AS x' in:\n$pairs") + } + test("describe extended promotes Comment and Collation to top-level rows") { // v1 `CatalogTable.toJsonLinkedHashMap` and v2 `DescribeV2ViewExec` both render Comment / // Collation as their own rows in the EXTENDED block, separately from the generic diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala index 384635857865..83bd04c2a9fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowViewsSuiteBase.scala @@ -44,4 +44,24 @@ trait ShowViewsSuiteBase extends QueryTest with DDLCommandTestUtils { assert(names.contains("show_views_match")) assert(!names.contains("show_views_skip")) } + + test("does not include non-view table entries") { + // SHOW VIEWS lists views and only views. Both v1 (session catalog routing through + // ShowTablesCommand-with-views-only) and v2 (`ShowViewsExec` routing through + // `ViewCatalog.listViews`) should exclude tables, and both must mark `isTemporary` as + // false for persistent view rows. + val viewName = "v_show_views_only" + val tableName = "t_not_in_show_views" + val table = s"$catalog.$namespace.$tableName" + sql(s"CREATE VIEW $catalog.$namespace.$viewName AS SELECT 1 AS x") + withTable(table) { + sql(s"CREATE TABLE $table (x INT) USING parquet") + val rows = sql(s"SHOW VIEWS IN $catalog.$namespace").collect() + val names = rows.map(_.getString(1)).toSet + assert(names.contains(viewName), s"$viewName missing from SHOW VIEWS: $names") + assert(!names.contains(tableName), s"non-view leaked into SHOW VIEWS: $names") + rows.foreach(r => assert(!r.getBoolean(2), + s"isTemporary must be false for persistent view rows: $r")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala index 94f9a47e704b..2dc1cef80d4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeViewSuite.scala @@ -23,19 +23,14 @@ class DescribeViewSuite extends command.DescribeViewSuiteBase with ViewCommandSuiteBase { test("V2: extended emits the v2-native `# Detailed View Information` header") { + // v1 emits `# Detailed Table Information` for views (CatalogTableType.VIEW shares the + // same describe path as CatalogTableType.{MANAGED,EXTERNAL}); v2's `DescribeV2ViewExec` + // routes views to a dedicated header. Pin the v2-side text here so the divergence stays + // intentional. val view = s"$catalog.$namespace.v2_desc_ext_header" sql(s"CREATE VIEW $view AS SELECT 1 AS x") val rows = sql(s"DESCRIBE TABLE EXTENDED $view").collect().map(_.getString(0)) assert(rows.contains("# Detailed View Information"), s"v2 extended describe should emit the View header; got:\n${rows.mkString("\n")}") } - - test("V2: extended block includes view text and catalog name") { - val view = s"$catalog.$namespace.v2_desc_ext_body" - sql(s"CREATE VIEW $view AS SELECT 7 AS x") - val rows = sql(s"DESCRIBE TABLE EXTENDED $view").collect() - val pairs = rows.map(r => r.getString(0) -> r.getString(1)).toMap - assert(pairs.get("Catalog").contains(catalog)) - assert(pairs.get("View Text").exists(_.contains("SELECT 7 AS x"))) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala index ac4c4b475866..a3e5533eeb3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowViewsSuite.scala @@ -18,34 +18,11 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, TableInfo, TableSummary} +import org.apache.spark.sql.connector.catalog.BasicInMemoryTableCatalog import org.apache.spark.sql.execution.command -import org.apache.spark.sql.types.StructType class ShowViewsSuite extends command.ShowViewsSuiteBase with ViewCommandSuiteBase { - test("V2: SHOW VIEWS does not include non-view table entries") { - sql(s"CREATE VIEW $catalog.$namespace.v_v2_only_views AS SELECT 1 AS x") - val tableIdent = Identifier.of(Array(namespace), "t_v2_not_in_show_views") - viewCatalog.createTable( - tableIdent, - new TableInfo.Builder() - .withSchema(new StructType().add("x", "int")) - .withTableType(TableSummary.EXTERNAL_TABLE_TYPE) - .build()) - try { - val rows = sql(s"SHOW VIEWS IN $catalog.$namespace").collect() - val names = rows.map(_.getString(1)).toSet - assert(names.contains("v_v2_only_views")) - assert(!names.contains("t_v2_not_in_show_views"), - s"non-view leaked into SHOW VIEWS: $names") - rows.foreach(r => assert(!r.getBoolean(2), - s"isTemporary must be false for v2 SHOW VIEWS: $r")) - } finally { - viewCatalog.dropTable(tableIdent) - } - } - test("V2: SHOW VIEWS on a non-ViewCatalog catalog fails") { withSQLConf( "spark.sql.catalog.no_view_show_cat" -> classOf[BasicInMemoryTableCatalog].getName) { From b16996c1f1944716f2af225d69453c75a8c66e80 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 15:37:10 +0000 Subject: [PATCH 14/17] [SPARK-56655][SQL] Make code comments self-contained Several comments in this PR's added code described what the code does in terms of "pre-PR behavior", "before the X change", "Post-migration", etc. That phrasing relies on the reader having the PR's history loaded as context; once the PR merges, a future maintainer reading the same comment has no way to recover the prior state being referenced. Rewrite each affected comment to describe what the code does *now*, citing the structural reason rather than the historical change: - `DescribeRelationJsonCommand.scala`: "preserves pre-PR behavior" -> "the command projects `ViewInfo` to a `CatalogTable` via `V1Table.toCatalogTable` so DESC ... AS JSON works uniformly across session and non-session view catalogs." - `views.scala` (`AlterViewAsCommand.collation`): "callers that omit this argument retain the pre-PR behavior" -> "callers that omit this argument keep the existing view's collation untouched." - `DataSourceV2MetadataViewSuite.scala` (4 comments): "Before the `fullIdent` change ...", "Before the error signatures took `Seq[String]` ...", "Pre-PR, `ResolvedPersistentView.metadata: CatalogTable` was always populated ...", "Post-migration the errors render the full multi-part name." -> all rewritten to describe the current routing (via `fullIdent` / `Seq[String]` / `V1Table.toCatalogTable`) and why it preserves multi-level-namespace segments in user-visible messages. No code changes -- comment-only. Co-authored-by: Isaac --- .../command/DescribeRelationJsonCommand.scala | 10 ++-- .../spark/sql/execution/command/views.scala | 13 +++--- .../DataSourceV2MetadataViewSuite.scala | 46 +++++++++++-------- 3 files changed, 37 insertions(+), 32 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala index 86d13a0bdd23..5f8528b67954 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala @@ -71,11 +71,11 @@ case class DescribeRelationJsonCommand( if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(v.identifier.name()) } - // For session-catalog (v1) views, recover the original `CatalogTable` from - // `V1ViewInfo`. For non-session v2 views (which carry a plain `ViewInfo`), synthesize - // an equivalent `CatalogTable` via `V1Table.toCatalogTable` -- mirrors the - // `CreateTableLike` strategy case in `DataSourceV2Strategy` and preserves pre-PR - // behavior of `DESC ... AS JSON` working on any persistent view. + // Resolve `v.info` to a `CatalogTable` so the JSON renderer below can read v1-shaped + // fields uniformly. Session-catalog views carry the original `CatalogTable` inside + // `V1ViewInfo`; non-session v2 views carry a plain `ViewInfo` and are projected to a + // `CatalogTable` via `V1Table.toCatalogTable`, the same conversion the + // `CreateTableLike` strategy case in `DataSourceV2Strategy` uses. val metadata = v.info match { case v1Info: V1ViewInfo => v1Info.v1Table case info => V1Table.toCatalogTable(v.catalog, v.identifier, info) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 62f271c05591..807342c0e90c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -236,13 +236,12 @@ case class AlterViewAsCommand( query: LogicalPlan, isAnalyzed: Boolean = false, referredTempFunctions: Seq[String] = Seq.empty, - // Analysis-time collation for the resolved view (after `ApplyDefaultCollation` has had a - // chance to fill the namespace default into a previously-empty `CatalogTable.collation`). - // `ResolveSessionCatalog` populates this from `ResolvedPersistentView.info.properties`'s - // `PROP_COLLATION`; only `alterPermanentView` consumes it. `None` means "no collation - // specified at analysis time" -- `alterPermanentView` then preserves the existing - // `CatalogTable.collation` so external constructors that omit this argument retain the - // pre-PR behavior. + // Analysis-time collation for the resolved view. `ApplyDefaultCollation` may have folded + // the namespace default into the resolved view's `CatalogTable.collation` if it was empty; + // `ResolveSessionCatalog` then reads `ResolvedPersistentView.info.properties`'s + // `PROP_COLLATION` and passes it here. Only `alterPermanentView` consumes it: `Some(x)` + // overwrites `CatalogTable.collation`, `None` falls through to the existing typed field + // so callers that omit this argument keep the existing view's collation untouched. collation: Option[String] = None) extends RunnableCommand with AnalysisOnlyCommand with CTEInChildren { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala index 0921e9e74781..cc22d01fe079 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala @@ -164,9 +164,11 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") // Two views whose last namespace segment collides (`inner`) but whose full multi-part - // identifiers differ. Before the `fullIdent` change both collapsed to - // `TableIdentifier(v, Some("inner"), Some("view_catalog"))` and cyclic detection would - // false-positive on a legitimate cross-namespace REPLACE. + // identifiers differ -- pin that cyclic detection compares them by `fullIdent` + // (`view_catalog.ns1.inner.v` vs `view_catalog.ns2.inner.v`) and not by the lossy + // 3-part `TableIdentifier` form, which would collapse both to + // `TableIdentifier(v, Some("inner"), Some("view_catalog"))` and false-positive on + // legitimate cross-namespace REPLACE. sql("CREATE VIEW view_catalog.ns1.inner.v AS SELECT x FROM spark_catalog.default.t") sql("CREATE VIEW view_catalog.ns2.inner.v AS " + "SELECT x FROM view_catalog.ns1.inner.v") @@ -191,9 +193,10 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { sql("CREATE VIEW view_catalog.ns1.inner.v_err AS " + "SELECT x FROM spark_catalog.default.t") // Second CREATE surfaces `viewAlreadyExistsError` (via TableAlreadyExistsException from - // the catalog). Before the error signatures took `Seq[String]`, `legacyName` collapsed - // ns1.inner into just `inner` and the error said `view_catalog.inner.v_err` -- missing - // the outer `ns1` segment. + // the catalog). Pin that the rendered name carries every multi-level-namespace segment + // (`view_catalog.ns1.inner.v_err`) -- routing the name through `Seq[String]` rather + // than a 3-part `TableIdentifier` is what preserves the outer `ns1` segment in the + // user-visible message. val dup = intercept[AnalysisException] { sql("CREATE VIEW view_catalog.ns1.inner.v_err AS " + "SELECT x FROM spark_catalog.default.t") @@ -245,9 +248,12 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { sql("CREATE VIEW view_catalog.ns2.inner.v_alter AS " + "SELECT x FROM view_catalog.ns1.inner.v_alter") - // Legitimate non-cyclic ALTER -- new body does not reference the altered view. Before - // `fullIdent` this false-positived because the two views collapsed to the same - // TableIdentifier(v_alter, Some("inner"), Some("view_catalog")). + // Legitimate non-cyclic ALTER -- the new body references `spark_catalog.default.t`, + // not the view being altered. Pin that ALTER's cyclic detection compares views by + // `fullIdent` so the two `inner.v_alter` views in different namespaces stay distinct; + // a comparison via the lossy 3-part `TableIdentifier` would collapse both to + // `TableIdentifier(v_alter, Some("inner"), Some("view_catalog"))` and false-positive + // here. sql("ALTER VIEW view_catalog.ns1.inner.v_alter AS " + "SELECT x FROM spark_catalog.default.t WHERE x > 1") checkAnswer( @@ -264,12 +270,12 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { } test("temp-object reference errors render the full multi-level namespace") { - // `verifyTemporaryObjectsNotExists` / `verifyAutoGeneratedAliasesNotExists` used to take a - // `TableIdentifier` built via `asLegacyTableIdentifier`, which collapses multi-level - // namespaces to the last segment -- so a temp-function reference on - // `view_catalog.ns1.inner.v_tempfn` produced an error naming - // `view_catalog.inner.v_tempfn` and dropped the `ns1` middle segment. Post-migration the - // errors render the full multi-part name. + // `verifyTemporaryObjectsNotExists` / `verifyAutoGeneratedAliasesNotExists` route the + // view name through `Seq[String]` rather than a 3-part `TableIdentifier`, so a + // temp-function reference inside `view_catalog.ns1.inner.v_tempfn` surfaces an error + // naming the full multi-part identifier. Routing through `asLegacyTableIdentifier` + // would collapse `ns1.inner` to the last segment and drop the outer `ns1` from the + // user-visible message. withTable("spark_catalog.default.t") { Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t") spark.udf.register("temp_udf_multi", (i: Int) => i + 1) @@ -339,11 +345,11 @@ class DataSourceV2MetadataViewSuite extends QueryTest with SharedSparkSession { } test("DESCRIBE TABLE EXTENDED ... AS JSON on a v2 view succeeds") { - // Pre-PR, `ResolvedPersistentView.metadata: CatalogTable` was always populated (a - // synthesized `CatalogTable` for v2 views), so DESC ... AS JSON worked uniformly. Post - // ResolvedPersistentView -> `info: ViewInfo`, the JSON command now synthesizes a - // `CatalogTable` via `V1Table.toCatalogTable` for non-session v2 views to preserve that - // pre-PR behavior. + // `DescribeRelationJsonCommand` is a v1 runnable command that reads v1-shaped fields off + // a `CatalogTable`. For non-session v2 views the resolved `ResolvedPersistentView.info` + // is a plain `ViewInfo`; the command projects it to a `CatalogTable` via + // `V1Table.toCatalogTable` so DESC ... AS JSON works uniformly across session and + // non-session view catalogs. seedV2View("v_desc_json") val rows = sql( "DESCRIBE TABLE EXTENDED view_catalog.default.v_desc_json AS JSON").collect() From 19b2d5b0f883856a1b5b8b73b03edd3a16f34f07 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Apr 2026 19:08:27 +0000 Subject: [PATCH 15/17] [SPARK-56655][SQL] Stable plan-tree rendering for ResolvedPersistentView; regen 9 analyzer-test goldens Two CI failures, both originating in this PR's `ResolvedPersistentView.metadata: CatalogTable` -> `ResolvedPersistentView.info: ViewInfo` migration: 1. `PlanResolutionSuite "alter view: alter view properties"` NPE. The Mockito mock for the "view" identifier in `createV1TableMock` did not stub the view-only fields. Mockito defaults unstubbed Object returns to `null`, so when v1 resolution wraps the mock in `V1ViewInfo(v1Table)` and `V1ViewInfo.builderFrom` calls `v1Table.viewText.getOrElse("")`, the call site receives a literal Java `null` (not `None`) and NPEs. Stub `viewText`, `viewCatalogAndNamespace`, `viewSQLConfigs`, `viewQueryColumnNames` for the VIEW case so the mock matches the shape a real `CatalogTable` would have. Real `CatalogTable`s never produce null Options because their case-class defaults are `None` / `Seq.empty` / etc., so no production-side defensive code is needed. 2. 9 `SQLQueryTestSuite *_analyzer_test` golden file mismatches. The case-class default `toString` rendered the new `info: ViewInfo` field via `Object.toString` -- producing `V1ViewInfo@` (and a similarly opaque hash for v2 plain-`ViewInfo`) -- non-deterministic between JVM runs, which is exactly what golden files cannot tolerate. Override `ResolvedPersistentView.stringArgs` so the third positional element renders as the qualified `catalog.namespace.name` string (via `quoted`), matching the spirit of the pre-migration `CatalogTable.toString` output but stable under hash randomization. Regenerate the 9 affected analyzer-test goldens via `SPARK_GENERATE_GOLDEN_FILES=1`; the only delta vs. master is the new stable rendering plus, for ALTER VIEW AS goldens, a trailing `` argument from the new `AlterViewAsCommand.collation` field added earlier in this PR. Co-authored-by: Isaac --- .../catalyst/analysis/v2ResolutionPlans.scala | 8 + .../analyzer-results/charvarchar.sql.out | 40 ++--- .../analyzer-results/describe.sql.out | 8 +- .../postgreSQL/create_view.sql.out | 137 +++++++++--------- .../view-schema-binding-config.sql.out | 28 ++-- .../view-schema-binding.sql.out | 14 +- .../view-schema-compensation.sql.out | 18 +-- .../view-schema-evolution.sql.out | 38 ++--- .../view-schema-type-evolution.sql.out | 24 +-- .../view-with-default-collation.sql.out | 6 +- .../command/PlanResolutionSuite.scala | 9 ++ 11 files changed, 161 insertions(+), 169 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 71e3c4d100dd..a8f5f0688890 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -250,6 +250,14 @@ case class ResolvedPersistentView( // it output does not affect query resolution. override lazy val output: Seq[Attribute] = toAttributes(CharVarcharUtils.replaceCharVarcharWithStringInSchema(info.schema)) + + // Render `info` in plan-tree output as the qualified view name. The default case-class + // `toString` would format `info` via `Object.toString`, which produces `V1ViewInfo@` + // for the v1 leg and a similarly opaque hash for the v2 leg -- non-deterministic and useless + // in EXPLAIN / golden file output. Replace it with the multi-part `catalog.namespace.name` + // form so EXPLAIN, plan-tree dumps, and `SQLQueryTestSuite` golden files remain stable. + override protected def stringArgs: Iterator[Any] = + Iterator(catalog, identifier, (catalog.name +: identifier.namespace :+ identifier.name).quoted) } /** diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out index db5d61ebcc44..09490b8a7fa9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out @@ -83,21 +83,18 @@ ShowCreateTable false, [createtab_stmt#x] -- !query create view char_view as select * from char_tbl -- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", - "sqlState" : "42P07", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`char_view`" - } -} +CreateViewCommand `spark_catalog`.`default`.`char_view`, select * from char_tbl, false, false, PersistedView, COMPENSATION, true + +- Project [c#x, v#x] + +- SubqueryAlias spark_catalog.default.char_tbl + +- Project [static_invoke(CharVarcharCodegenUtils.readSidePadding(c#x, 5)) AS c#x, v#x] + +- Relation spark_catalog.default.char_tbl[c#x,v#x] parquet -- !query desc formatted char_view -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, `spark_catalog`.`default`.`char_view` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, spark_catalog.default.char_view -- !query @@ -189,7 +186,7 @@ AlterViewAsCommand `spark_catalog`.`default`.`char_view`, select * from char_tbl desc formatted char_view -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, `spark_catalog`.`default`.`char_view` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, spark_catalog.default.char_view -- !query @@ -215,7 +212,7 @@ AlterTableSetPropertiesCommand `spark_catalog`.`default`.`char_view`, [yes=no], desc formatted char_view -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, `spark_catalog`.`default`.`char_view` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, spark_catalog.default.char_view -- !query @@ -241,7 +238,7 @@ AlterTableUnsetPropertiesCommand `spark_catalog`.`default`.`char_view`, [yes], f desc formatted char_view -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, `spark_catalog`.`default`.`char_view` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.char_view, spark_catalog.default.char_view -- !query @@ -260,14 +257,7 @@ DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, d -- !query create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2) -- !query analysis -org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException -{ - "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", - "sqlState" : "42P07", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`char_part`" - } -} +CreateDataSourceTableCommand `spark_catalog`.`default`.`char_part`, false -- !query @@ -292,15 +282,7 @@ AlterTableChangeColumnCommand `spark_catalog`.`default`.`char_part`, v1, StructF -- !query alter table char_part add partition (v2='ke', c2='nt') location 'loc1' -- !query analysis -org.apache.spark.sql.catalyst.analysis.PartitionsAlreadyExistException -{ - "errorClass" : "PARTITIONS_ALREADY_EXIST", - "sqlState" : "428FT", - "messageParameters" : { - "partitionList" : "PARTITION (`v2` = ke, `c2` = nt)", - "tableName" : "`default`.`char_part`" - } -} +AlterTableAddPartitionCommand `spark_catalog`.`default`.`char_part`, [(Map(v2 -> ke, c2 -> nt),Some(loc1))], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out index f284b4cabdb4..3519577e2bef 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -313,28 +313,28 @@ org.apache.spark.sql.AnalysisException DESC v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query DESC TABLE v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query DESC FORMATTED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query DESC EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out index 6d159f4c9ee0..c6473ae4fd2c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out @@ -38,14 +38,7 @@ DropTable false, false -- !query CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) USING PARQUET -- !query analysis -org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException -{ - "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", - "sqlState" : "42P07", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`view_base_table`" - } -} +CreateDataSourceTableCommand `spark_catalog`.`default`.`view_base_table`, false -- !query @@ -262,7 +255,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v1`, SELECT * FROM base_tabl DESC TABLE EXTENDED v1 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v1`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v1, `spark_catalog`.`temp_view_test`.`v1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v1, spark_catalog.temp_view_test.v1 -- !query @@ -310,7 +303,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v2`, SELECT * FROM base_tabl DESC TABLE EXTENDED temp_view_test.v2 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v2`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v2, `spark_catalog`.`temp_view_test`.`v2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v2, spark_catalog.temp_view_test.v2 -- !query @@ -353,7 +346,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v3`, SELECT t1.a AS t1_a, t2 DESC TABLE EXTENDED v3 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v3`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v3, `spark_catalog`.`temp_view_test`.`v3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v3, spark_catalog.temp_view_test.v3 -- !query @@ -411,7 +404,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v4`, SELECT * FROM base_tabl DESC TABLE EXTENDED v4 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v4`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v4, `spark_catalog`.`temp_view_test`.`v4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v4, spark_catalog.temp_view_test.v4 -- !query @@ -433,7 +426,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v5`, SELECT t1.id, t2.a FROM DESC TABLE EXTENDED v5 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v5`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v5, `spark_catalog`.`temp_view_test`.`v5` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v5, spark_catalog.temp_view_test.v5 -- !query @@ -453,7 +446,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v6`, SELECT * FROM base_tabl DESC TABLE EXTENDED v6 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v6`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v6, `spark_catalog`.`temp_view_test`.`v6` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v6, spark_catalog.temp_view_test.v6 -- !query @@ -473,7 +466,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v7`, SELECT * FROM base_tabl DESC TABLE EXTENDED v7 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v7`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v7, `spark_catalog`.`temp_view_test`.`v7` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v7, spark_catalog.temp_view_test.v7 -- !query @@ -492,7 +485,7 @@ CreateViewCommand `spark_catalog`.`temp_view_test`.`v8`, SELECT * FROM base_tabl DESC TABLE EXTENDED v8 -- !query analysis DescribeTableCommand `spark_catalog`.`temp_view_test`.`v8`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v8, `spark_catalog`.`temp_view_test`.`v8` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), temp_view_test.v8, spark_catalog.temp_view_test.v8 -- !query @@ -675,7 +668,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp1`, SELECT * FROM t1 C DESC TABLE EXTENDED nontemp1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp1`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp1, `spark_catalog`.`testviewschm2`.`nontemp1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp1, spark_catalog.testviewschm2.nontemp1 -- !query @@ -710,7 +703,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp2`, SELECT * FROM t1 I DESC TABLE EXTENDED nontemp2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp2`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp2, `spark_catalog`.`testviewschm2`.`nontemp2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp2, spark_catalog.testviewschm2.nontemp2 -- !query @@ -745,7 +738,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp3`, SELECT * FROM t1 L DESC TABLE EXTENDED nontemp3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp3`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp3, `spark_catalog`.`testviewschm2`.`nontemp3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp3, spark_catalog.testviewschm2.nontemp3 -- !query @@ -780,7 +773,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp4`, SELECT * FROM t1 L DESC TABLE EXTENDED nontemp4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp4`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp4, `spark_catalog`.`testviewschm2`.`nontemp4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.nontemp4, spark_catalog.testviewschm2.nontemp4 -- !query @@ -885,7 +878,7 @@ AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f), false, false, DESC TABLE EXTENDED pubview -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`pubview`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.pubview, `spark_catalog`.`testviewschm2`.`pubview` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.pubview, spark_catalog.testviewschm2.pubview -- !query @@ -928,7 +921,7 @@ AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j), false DESC TABLE EXTENDED mytempview -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`mytempview`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.mytempview, `spark_catalog`.`testviewschm2`.`mytempview` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.mytempview, spark_catalog.testviewschm2.mytempview -- !query @@ -1074,28 +1067,28 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, select * fro DESC TABLE aliased_view_1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, `spark_catalog`.`testviewschm2`.`aliased_view_1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, spark_catalog.testviewschm2.aliased_view_1 -- !query DESC TABLE aliased_view_2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, `spark_catalog`.`testviewschm2`.`aliased_view_2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, spark_catalog.testviewschm2.aliased_view_2 -- !query DESC TABLE aliased_view_3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, `spark_catalog`.`testviewschm2`.`aliased_view_3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, spark_catalog.testviewschm2.aliased_view_3 -- !query DESC TABLE aliased_view_4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, `spark_catalog`.`testviewschm2`.`aliased_view_4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, spark_catalog.testviewschm2.aliased_view_4 -- !query @@ -1108,28 +1101,28 @@ AlterTableRenameCommand `spark_catalog`.`testviewschm2`.`tx1`, `a1`, false DESC TABLE aliased_view_1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, `spark_catalog`.`testviewschm2`.`aliased_view_1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, spark_catalog.testviewschm2.aliased_view_1 -- !query DESC TABLE aliased_view_2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, `spark_catalog`.`testviewschm2`.`aliased_view_2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, spark_catalog.testviewschm2.aliased_view_2 -- !query DESC TABLE aliased_view_3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, `spark_catalog`.`testviewschm2`.`aliased_view_3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, spark_catalog.testviewschm2.aliased_view_3 -- !query DESC TABLE aliased_view_4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, `spark_catalog`.`testviewschm2`.`aliased_view_4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, spark_catalog.testviewschm2.aliased_view_4 -- !query @@ -1142,28 +1135,28 @@ AlterTableRenameCommand `spark_catalog`.`testviewschm2`.`tt1`, `a2`, false DESC TABLE aliased_view_1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, `spark_catalog`.`testviewschm2`.`aliased_view_1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, spark_catalog.testviewschm2.aliased_view_1 -- !query DESC TABLE aliased_view_2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, `spark_catalog`.`testviewschm2`.`aliased_view_2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, spark_catalog.testviewschm2.aliased_view_2 -- !query DESC TABLE aliased_view_3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, `spark_catalog`.`testviewschm2`.`aliased_view_3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, spark_catalog.testviewschm2.aliased_view_3 -- !query DESC TABLE aliased_view_4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, `spark_catalog`.`testviewschm2`.`aliased_view_4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, spark_catalog.testviewschm2.aliased_view_4 -- !query @@ -1176,28 +1169,28 @@ AlterTableRenameCommand `spark_catalog`.`testviewschm2`.`a1`, `tt1`, false DESC TABLE aliased_view_1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, `spark_catalog`.`testviewschm2`.`aliased_view_1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_1, spark_catalog.testviewschm2.aliased_view_1 -- !query DESC TABLE aliased_view_2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, `spark_catalog`.`testviewschm2`.`aliased_view_2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_2, spark_catalog.testviewschm2.aliased_view_2 -- !query DESC TABLE aliased_view_3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, `spark_catalog`.`testviewschm2`.`aliased_view_3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_3, spark_catalog.testviewschm2.aliased_view_3 -- !query DESC TABLE aliased_view_4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, `spark_catalog`.`testviewschm2`.`aliased_view_4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.aliased_view_4, spark_catalog.testviewschm2.aliased_view_4 -- !query @@ -1334,35 +1327,35 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`v3`, select * from tt2 join t DESC TABLE v1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1, `spark_catalog`.`testviewschm2`.`v1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1, spark_catalog.testviewschm2.v1 -- !query DESC TABLE v1a -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1a`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1a, `spark_catalog`.`testviewschm2`.`v1a` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1a, spark_catalog.testviewschm2.v1a -- !query DESC TABLE v2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2, `spark_catalog`.`testviewschm2`.`v2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2, spark_catalog.testviewschm2.v2 -- !query DESC TABLE v2a -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2a`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2a, `spark_catalog`.`testviewschm2`.`v2a` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2a, spark_catalog.testviewschm2.v2a -- !query DESC TABLE v3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v3, `spark_catalog`.`testviewschm2`.`v3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v3, spark_catalog.testviewschm2.v3 -- !query @@ -1381,35 +1374,35 @@ AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt2`, [StructField( DESC TABLE v1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1, `spark_catalog`.`testviewschm2`.`v1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1, spark_catalog.testviewschm2.v1 -- !query DESC TABLE v1a -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1a`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1a, `spark_catalog`.`testviewschm2`.`v1a` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1a, spark_catalog.testviewschm2.v1a -- !query DESC TABLE v2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2, `spark_catalog`.`testviewschm2`.`v2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2, spark_catalog.testviewschm2.v2 -- !query DESC TABLE v2a -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2a`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2a, `spark_catalog`.`testviewschm2`.`v2a` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2a, spark_catalog.testviewschm2.v2a -- !query DESC TABLE v3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v3, `spark_catalog`.`testviewschm2`.`v3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v3, spark_catalog.testviewschm2.v3 -- !query @@ -1441,35 +1434,35 @@ AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt3`, [StructField( DESC TABLE v1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1, `spark_catalog`.`testviewschm2`.`v1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1, spark_catalog.testviewschm2.v1 -- !query DESC TABLE v1a -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1a`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1a, `spark_catalog`.`testviewschm2`.`v1a` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v1a, spark_catalog.testviewschm2.v1a -- !query DESC TABLE v2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2, `spark_catalog`.`testviewschm2`.`v2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2, spark_catalog.testviewschm2.v2 -- !query DESC TABLE v2a -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2a`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2a, `spark_catalog`.`testviewschm2`.`v2a` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v2a, spark_catalog.testviewschm2.v2a -- !query DESC TABLE v3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`v3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v3, `spark_catalog`.`testviewschm2`.`v3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.v3, spark_catalog.testviewschm2.v3 -- !query @@ -1502,7 +1495,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`vv1`, select * from (tt5 cros DESC TABLE vv1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv1, `spark_catalog`.`testviewschm2`.`vv1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv1, spark_catalog.testviewschm2.vv1 -- !query @@ -1515,7 +1508,7 @@ AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt5`, [StructField( DESC TABLE vv1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv1, `spark_catalog`.`testviewschm2`.`vv1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv1, spark_catalog.testviewschm2.vv1 -- !query @@ -1528,7 +1521,7 @@ AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt5`, [StructField( DESC TABLE vv1 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv1`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv1, `spark_catalog`.`testviewschm2`.`vv1` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv1, spark_catalog.testviewschm2.vv1 -- !query @@ -1574,7 +1567,7 @@ select * from tt7 full join tt8 using (x), tt8 tt8x, false, false, PersistedView DESC TABLE vv2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv2, `spark_catalog`.`testviewschm2`.`vv2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv2, spark_catalog.testviewschm2.vv2 -- !query @@ -1617,7 +1610,7 @@ select * from DESC TABLE vv3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv3, `spark_catalog`.`testviewschm2`.`vv3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv3, spark_catalog.testviewschm2.vv3 -- !query @@ -1665,7 +1658,7 @@ select * from DESC TABLE vv4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv4`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv4, `spark_catalog`.`testviewschm2`.`vv4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv4, spark_catalog.testviewschm2.vv4 -- !query @@ -1690,21 +1683,21 @@ AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt8`, [StructField( DESC TABLE vv2 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv2`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv2, `spark_catalog`.`testviewschm2`.`vv2` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv2, spark_catalog.testviewschm2.vv2 -- !query DESC TABLE vv3 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv3`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv3, `spark_catalog`.`testviewschm2`.`vv3` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv3, spark_catalog.testviewschm2.vv3 -- !query DESC TABLE vv4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv4`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv4, `spark_catalog`.`testviewschm2`.`vv4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv4, spark_catalog.testviewschm2.vv4 -- !query @@ -1751,14 +1744,14 @@ select * from tt7a left join tt8a using (x), tt8a tt8ax, false, false, Persisted DESC TABLE vv4 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv4`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv4, `spark_catalog`.`testviewschm2`.`vv4` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv4, spark_catalog.testviewschm2.vv4 -- !query DESC TABLE vv2a -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv2a`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv2a, `spark_catalog`.`testviewschm2`.`vv2a` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv2a, spark_catalog.testviewschm2.vv2a -- !query @@ -1790,14 +1783,14 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`vv5`, select x,y,z from tt9 j DESC TABLE vv5 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv5`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv5, `spark_catalog`.`testviewschm2`.`vv5` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv5, spark_catalog.testviewschm2.vv5 -- !query DESC TABLE vv5 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv5`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv5, `spark_catalog`.`testviewschm2`.`vv5` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv5, spark_catalog.testviewschm2.vv5 -- !query @@ -1841,7 +1834,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`vv6`, select x,y,z,q from DESC TABLE vv6 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv6`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv6, `spark_catalog`.`testviewschm2`.`vv6` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv6, spark_catalog.testviewschm2.vv6 -- !query @@ -1854,7 +1847,7 @@ AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt11`, [StructField DESC TABLE vv6 -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv6`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv6, `spark_catalog`.`testviewschm2`.`vv6` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.vv6, spark_catalog.testviewschm2.vv6 -- !query @@ -1887,7 +1880,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`tt18v`, select * from int8_tb DESC TABLE tt18v -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt18v`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt18v, `spark_catalog`.`testviewschm2`.`tt18v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt18v, spark_catalog.testviewschm2.tt18v -- !query @@ -1908,7 +1901,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`tt21v`, select * from tt5 nat DESC TABLE tt21v -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt21v`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt21v, `spark_catalog`.`testviewschm2`.`tt21v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt21v, spark_catalog.testviewschm2.tt21v -- !query @@ -1929,7 +1922,7 @@ CreateViewCommand `spark_catalog`.`testviewschm2`.`tt22v`, select * from tt5 nat DESC TABLE tt22v -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt22v`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt22v, `spark_catalog`.`testviewschm2`.`tt22v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt22v, spark_catalog.testviewschm2.tt22v -- !query @@ -1954,7 +1947,7 @@ select 42, 43, false, false, PersistedView, COMPENSATION, true DESC TABLE tt23v -- !query analysis DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt23v`, false, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt23v, `spark_catalog`.`testviewschm2`.`tt23v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), testviewschm2.tt23v, spark_catalog.testviewschm2.tt23v -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding-config.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding-config.sql.out index 73b266fb02f9..0915e3db57cd 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding-config.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding-config.sql.out @@ -111,7 +111,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, SELECT 1, false, true, Persiste DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -197,7 +197,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -235,7 +235,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -294,7 +294,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -344,7 +344,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -397,7 +397,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -443,7 +443,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -489,7 +489,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -551,7 +551,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -600,7 +600,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -637,7 +637,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -674,7 +674,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -707,7 +707,7 @@ SetCommand (spark.sql.legacy.viewSchemaBindingMode,Some(true)) DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -732,7 +732,7 @@ SetCommand (spark.sql.legacy.viewSchemaCompensation,Some(true)) DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding.sql.out index e4292334e784..c91e3d3ee6de 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-binding.sql.out @@ -37,7 +37,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -73,7 +73,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -114,7 +114,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -151,7 +151,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -198,7 +198,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -211,7 +211,7 @@ AlterViewSchemaBindingCommand `spark_catalog`.`default`.`v`, BINDING DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -247,7 +247,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-compensation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-compensation.sql.out index bf05d5cd617a..15210983b7d8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-compensation.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-compensation.sql.out @@ -43,7 +43,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -83,7 +83,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -123,7 +123,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -185,7 +185,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -234,7 +234,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -271,7 +271,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -308,7 +308,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -345,7 +345,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, SELECT * FROM t, false, true, P DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -395,7 +395,7 @@ AlterViewSchemaBindingCommand `spark_catalog`.`default`.`v`, COMPENSATION DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-evolution.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-evolution.sql.out index 36b2e04df0f8..c52181dbc360 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-evolution.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-evolution.sql.out @@ -44,7 +44,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -83,7 +83,7 @@ Project [c4#x, c5#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -122,7 +122,7 @@ Project [c4#x, c5#x, c6#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -170,7 +170,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -201,7 +201,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -250,7 +250,7 @@ Project [a1#x, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -290,7 +290,7 @@ Project [a1#x, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -330,7 +330,7 @@ Project [a1#x, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -379,7 +379,7 @@ Project [a1#x, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -416,7 +416,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -453,7 +453,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -482,7 +482,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, [(a1,None), (a2,None)], SELECT DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -498,7 +498,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, [(a1,None), (a2,None)], SELECT DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -530,7 +530,7 @@ Project [a1#xL, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -546,7 +546,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, [(a1,Some(a1)), (a2,Some(a2))], DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -578,7 +578,7 @@ Project [a1#xL, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -594,7 +594,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, SELECT * FROM t, false, true, P DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -625,7 +625,7 @@ Project [c1#xL, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -784,7 +784,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-type-evolution.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-type-evolution.sql.out index f097ae082546..05480903885f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-type-evolution.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/view-schema-type-evolution.sql.out @@ -45,7 +45,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -85,7 +85,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -125,7 +125,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -174,7 +174,7 @@ Project [c1#x, c2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -211,7 +211,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -248,7 +248,7 @@ org.apache.spark.sql.AnalysisException DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -277,7 +277,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, [(a1,None), (a2,None)], SELECT DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -309,7 +309,7 @@ Project [a1#xL, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -325,7 +325,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, [(a1,Some(a1)), (a2,Some(a2))], DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -357,7 +357,7 @@ Project [a1#xL, a2#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -427,7 +427,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query @@ -452,7 +452,7 @@ Project [c1#x] DESCRIBE EXTENDED v -- !query analysis DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] -+- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, `spark_catalog`.`default`.`v` ++- ResolvedPersistentView V2SessionCatalog(spark_catalog), default.v, spark_catalog.default.v -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/view-with-default-collation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/view-with-default-collation.sql.out index 02f40446a283..a1257ab482c6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/view-with-default-collation.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/view-with-default-collation.sql.out @@ -465,7 +465,7 @@ CreateViewCommand `spark_catalog`.`default`.`v`, UTF8_LCASE, SELECT 1, false, fa -- !query ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2 -- !query analysis -AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT 'a' AS c1, 'b' AS c2, true +AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT 'a' AS c1, 'b' AS c2, true, UTF8_LCASE +- Project [a AS c1#x, b AS c2#x] +- OneRowRelation @@ -495,7 +495,7 @@ Project [collation(c2#x) AS collation(c2)#x] -- !query ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A' -- !query analysis -AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT 'c' AS c3 WHERE 'a' = 'A', true +AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT 'c' AS c3 WHERE 'a' = 'A', true, UTF8_LCASE +- Project [c AS c3#x] +- Filter (a = A) +- OneRowRelation @@ -556,7 +556,7 @@ ALTER VIEW v AS -- !query analysis AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 FROM t - WHERE c1 = 'A' AND 'ć' = 'Č', true + WHERE c1 = 'A' AND 'ć' = 'Č', true, sr_CI_AI +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index a4b2ef49f0a2..d31f69dcfdd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -146,6 +146,15 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { when(t.properties).thenReturn(Map.empty) when(t.comment).thenReturn(None) when(t.collation).thenReturn(None) + if (tableType == CatalogTableType.VIEW) { + // Stub the view-only fields that resolution reads through `V1ViewInfo.builderFrom`. + // Mockito returns `null` for unstubbed Object methods, which would NPE the moment + // builderFrom calls `.getOrElse` / `.asJava` / `.toArray` on a null Option/Seq/Map. + when(t.viewText).thenReturn(None) + when(t.viewCatalogAndNamespace).thenReturn(Seq.empty) + when(t.viewSQLConfigs).thenReturn(Map.empty) + when(t.viewQueryColumnNames).thenReturn(Seq.empty) + } V1Table(t) } From 83a38b414e6e57b12bc40e58b254e3cca91b9ce5 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 30 Apr 2026 00:21:06 +0000 Subject: [PATCH 16/17] [SPARK-56655][SQL][TESTS] Fix analyzer-test goldens for shared-catalog state The previous regen in 19b2d5b0f88 captured fresh-catalog plan outputs for 4 queries that, under SQLQueryTestSuite's shared warehouse, actually resolve to *_ALREADY_EXISTS errors when the analyzer-test runs after the regular test. Re-regenerate those goldens with both tests in scope so the captured outputs match CI behavior; the legitimate ResolvedPersistentView rendering changes from this PR are preserved. Co-authored-by: Isaac --- .../analyzer-results/charvarchar.sql.out | 32 +++++++++++++++---- .../postgreSQL/create_view.sql.out | 9 +++++- 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out index 09490b8a7fa9..8915835fffe8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out @@ -83,11 +83,14 @@ ShowCreateTable false, [createtab_stmt#x] -- !query create view char_view as select * from char_tbl -- !query analysis -CreateViewCommand `spark_catalog`.`default`.`char_view`, select * from char_tbl, false, false, PersistedView, COMPENSATION, true - +- Project [c#x, v#x] - +- SubqueryAlias spark_catalog.default.char_tbl - +- Project [static_invoke(CharVarcharCodegenUtils.readSidePadding(c#x, 5)) AS c#x, v#x] - +- Relation spark_catalog.default.char_tbl[c#x,v#x] parquet +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`char_view`" + } +} -- !query @@ -257,7 +260,14 @@ DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, d -- !query create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2) -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`char_part`, false +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`char_part`" + } +} -- !query @@ -282,7 +292,15 @@ AlterTableChangeColumnCommand `spark_catalog`.`default`.`char_part`, v1, StructF -- !query alter table char_part add partition (v2='ke', c2='nt') location 'loc1' -- !query analysis -AlterTableAddPartitionCommand `spark_catalog`.`default`.`char_part`, [(Map(v2 -> ke, c2 -> nt),Some(loc1))], false +org.apache.spark.sql.catalyst.analysis.PartitionsAlreadyExistException +{ + "errorClass" : "PARTITIONS_ALREADY_EXIST", + "sqlState" : "428FT", + "messageParameters" : { + "partitionList" : "PARTITION (`v2` = ke, `c2` = nt)", + "tableName" : "`default`.`char_part`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out index c6473ae4fd2c..95627092dca2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out @@ -38,7 +38,14 @@ DropTable false, false -- !query CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) USING PARQUET -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`view_base_table`, false +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`view_base_table`" + } +} -- !query From b7e8fba44015427177cbc35d7da27fb3fac59bf8 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 30 Apr 2026 00:28:10 +0000 Subject: [PATCH 17/17] [SPARK-56655][SQL] Fix Java linter line-length in MetadataTable.java --- .../org/apache/spark/sql/connector/catalog/MetadataTable.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataTable.java index 0a8c29b29792..41107861023d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataTable.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataTable.java @@ -36,7 +36,8 @@ * {@link ViewInfo.Builder} (for views). A {@code MetadataTable} wrapping a * {@link TableInfo} can be returned from {@link TableCatalog#loadTable(Identifier)} for a * data-source table; a {@code MetadataTable} wrapping a {@link ViewInfo} can be returned - * from {@link TableViewCatalog#loadTableOrView(Identifier)} as the single-RPC perf opt-in for a view. + * from {@link TableViewCatalog#loadTableOrView(Identifier)} as the single-RPC perf opt-in + * for a view. * Downstream consumers distinguish the two by checking * {@code getTableInfo() instanceof ViewInfo}. *