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..41107861023d 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,18 @@ * (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 +52,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 82% 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..c3298831ef3e 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. * *
* 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 +167,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 +183,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 +192,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 +205,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 +221,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..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 @@ -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) @@ -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 da82de01f8e4..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 @@ -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}.
*
@@ -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 323a7db9c7ad..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
@@ -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,15 +1151,15 @@ 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))
+ 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..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
@@ -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.{CatalogV2Util, SupportsNamespaces, TableCatalog, V1ViewInfo, ViewInfo}
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,18 +207,44 @@ object ApplyDefaultCollation extends Rule[LogicalPlan] {
newCreateView
// We match against ResolvedPersistentView because temporary views don't have a
- // schema/catalog.
- case alterViewAs@AlterViewAs(resolvedPersistentView@ResolvedPersistentView(
- catalog: SupportsNamespaces, identifier, _), _, _, _, _)
- if resolvedPersistentView.metadata.collation.isEmpty =>
- val newResolvedPersistentView = resolvedPersistentView.copy(
- metadata = resolvedPersistentView.metadata.copy(
- collation = getCollationFromSchemaMetadata(catalog, identifier.namespace())))
- val newAlterViewAs = CurrentOrigin.withOrigin(alterViewAs.origin) {
- alterViewAs.copy(child = newResolvedPersistentView)
+ // schema/catalog. The rewrite covers both v1 (session-catalog, [[V1ViewInfo]]) and
+ // 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 =>
+ // 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. 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
}
- newAlterViewAs.copyTagsFrom(alterViewAs)
- newAlterViewAs
case createUserDefinedFunction@CreateUserDefinedFunction(
ResolvedIdentifier(catalog: SupportsNamespaces, identifier),
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/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
index 046acd20a9e3..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
@@ -227,15 +227,37 @@ 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))
+
+ // 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@` would silently destroy the table's entry.
- val catalog = spark.sessionState.catalogManager.catalog("view_catalog")
- .asInstanceOf[TestingRelationCatalog]
- 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"))
- }
- }
-
- // --- SHOW TABLES / SHOW VIEWS on a v2 catalog --------------------------------
-
- private def seedV2Table(name: String): Unit = {
- val catalog = spark.sessionState.catalogManager.catalog("view_catalog")
- .asInstanceOf[TestingRelationCatalog]
- catalog.createTable(
- Identifier.of(Array("default"), name),
- new TableInfo.Builder()
- .withSchema(new StructType().add("x", "int"))
- .withTableType(TableSummary.EXTERNAL_TABLE_TYPE)
- .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.
- 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")
- 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")
- }
- }
-}
-
-/**
- * A [[RelationCatalog]]: round-trips [[MetadataOnlyTable]] 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
- * [[TableCatalog#loadTable]] contract.
- */
-class TestingRelationCatalog extends RelationCatalog {
-
- // 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.
- 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.
- private def fixtureView(ident: Identifier): Option[ViewInfo] = ident.name() match {
- case "test_view" =>
- Some(new ViewInfo.Builder()
- .withSchema(new StructType().add("col", "string").add("i", "int"))
- .withQueryText(
- "SELECT col, col::int AS i FROM spark_catalog.default.t WHERE col = 'b'")
- .withSqlConfigs(java.util.Collections.singletonMap(
- SQLConf.ANSI_ENABLED.key, (ident.namespace().head == "ansi").toString))
- .build())
- case "test_unqualified_view" =>
- Some(new ViewInfo.Builder()
- .withSchema(new StructType().add("col", "string"))
- .withQueryText("SELECT col FROM t WHERE col = 'b'")
- .withCurrentCatalog("spark_catalog")
- .withCurrentNamespace(Array("default"))
- .build())
- case "test_unqualified_multi" =>
- // View whose captured catalog+namespace is view_catalog.ns1.ns2 (two-part). The
- // unqualified `t` in the body must resolve via that captured context to
- // view_catalog.ns1.ns2.t, which this catalog also serves (see `t` case below).
- Some(new ViewInfo.Builder()
- .withSchema(new StructType().add("col", "string"))
- .withQueryText("SELECT col FROM t")
- .withCurrentCatalog("view_catalog")
- .withCurrentNamespace(Array("ns1", "ns2"))
- .build())
- case "t" if ident.namespace().toSeq == Seq("ns1", "ns2") =>
- // Target of test_unqualified_multi's unqualified reference. Self-contained view so
- // the test doesn't need external data.
- Some(new ViewInfo.Builder()
- .withSchema(new StructType().add("col", "string"))
- .withQueryText("SELECT 'multi' AS col")
- .build())
- case _ => None
- }
-
- override def loadRelation(ident: Identifier): Table = {
- // Single-RPC perf path: returns tables AND views (as MetadataOnlyTable). 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.
- val key = (ident.namespace().toSeq, ident.name())
- Option(createdViews.get(key))
- .orElse(fixtureView(ident))
- .map(new MetadataOnlyTable(_, ident.toString))
- .getOrElse(throw new NoSuchTableException(ident))
- }
-
- override def createTable(ident: Identifier, info: TableInfo): Table = {
- // Mixed-catalog contract: createTable rejects when a view sits at ident with
- // TableAlreadyExistsException. The shared `createdViews` keyspace makes `putIfAbsent`
- // throw uniformly for both table-at-ident and view-at-ident collisions.
- val key = (ident.namespace().toSeq, ident.name())
- if (createdViews.putIfAbsent(key, info) != null) {
- throw new TableAlreadyExistsException(ident)
- }
- new MetadataOnlyTable(info, ident.toString)
- }
-
- /** Test-only accessor: returns the stored TableInfo (table or view) for the identifier. */
- def getStoredInfo(namespace: Array[String], name: String): TableInfo = {
- Option(createdViews.get((namespace.toSeq, name))).getOrElse {
- throw new NoSuchTableException(Identifier.of(namespace, name))
- }
- }
-
- /** Test-only accessor: returns the stored ViewInfo; fails if the entry 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")
- }
-
- override def alterTable(ident: Identifier, changes: TableChange*): Table = {
- throw new RuntimeException("shouldn't be called")
- }
- override def dropTable(ident: Identifier): Boolean = {
- val key = (ident.namespace().toSeq, ident.name())
- val existing = createdViews.get(key)
- if (existing == null || existing.isInstanceOf[ViewInfo]) return false
- createdViews.remove(key) != null
- }
- override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = {
- throw new RuntimeException("shouldn't be called")
- }
- override def listTables(namespace: Array[String]): Array[Identifier] = {
- // Tables only -- views are listed via ViewCatalog.listViews per the new contract.
- val targetNs = namespace.toSeq
- val ids = new java.util.ArrayList[Identifier]()
- createdViews.forEach { (key, info) =>
- if (key._1 == targetNs && !info.isInstanceOf[ViewInfo]) {
- ids.add(Identifier.of(key._1.toArray, key._2))
- }
- }
- ids.toArray(new Array[Identifier](0))
- }
-
- // ViewCatalog methods. Storage is shared with TableCatalog (mixed-catalog pattern).
-
- override def listViews(namespace: Array[String]): Array[Identifier] = {
- val targetNs = namespace.toSeq
- val ids = new java.util.ArrayList[Identifier]()
- createdViews.forEach { (key, info) =>
- if (key._1 == targetNs && 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 (createdViews.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 = createdViews.get(key)
- if (existing == null || !existing.isInstanceOf[ViewInfo]) {
- throw new NoSuchViewException(ident)
- }
- createdViews.put(key, info)
- info
- }
-
- override def dropView(ident: Identifier): Boolean = {
- val key = (ident.namespace().toSeq, ident.name())
- val existing = createdViews.get(key)
- if (existing == null || !existing.isInstanceOf[ViewInfo]) return false
- createdViews.remove(key) != null
- }
-
- private var catalogName = ""
- override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {
- catalogName = name
- }
- override def name(): String = catalogName
-}
-
-/**
- * A v2 catalog that does not implement ViewCatalog. Used by capability-gate tests: the gate
- * fires in `Analyzer.lookupTableOrView(viewOnly=true)` for ALTER VIEW and in
- * [[CheckViewReferences]] for CREATE VIEW -- in both cases before `loadTable` is called --
- * so this catalog's content is intentionally empty.
- */
-class TestingTableOnlyCatalog extends TableCatalog {
- override def loadTable(ident: Identifier): Table = throw new NoSuchTableException(ident)
-
- override def alterTable(ident: Identifier, changes: TableChange*): Table =
- throw new RuntimeException("shouldn't be called")
- override def dropTable(ident: Identifier): Boolean = false
- override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit =
- throw new RuntimeException("shouldn't be called")
- override def listTables(namespace: Array[String]): Array[Identifier] = Array.empty
- private var catalogName = ""
- override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {
- catalogName = name
- }
- override def name(): String = catalogName
-}
-
-/**
- * A pure [[ViewCatalog]] (no [[TableCatalog]] mixin). Used to exercise that the analyzer's
- * resolution paths skip the `loadTable` step and fall through to `loadView` for catalogs that
- * cannot host tables. Pre-seeds a single mutable view at `default.pure_v` so the read and
- * ALTER VIEW tests can both reach it.
- */
-class TestingViewOnlyCatalog extends ViewCatalog {
- private val store =
- new java.util.concurrent.ConcurrentHashMap[(Seq[String], String), ViewInfo]()
-
- // Seeded on first `initialize`. Filters `spark_catalog.default.t` so the read test can
- // assert deterministic output. ALTER VIEW tests overwrite it via `replaceView`.
- private def seedDefault(): Unit = {
- val key = (Seq("default"), "pure_v")
- if (!store.containsKey(key)) {
- val info = new ViewInfo.Builder()
- .withSchema(new StructType().add("x", "int"))
- .withQueryText("SELECT x FROM spark_catalog.default.t WHERE x > 1")
- .build()
- store.put(key, info)
- }
- }
-
- override def listViews(namespace: Array[String]): Array[Identifier] = {
- val target = namespace.toSeq
- val ids = new java.util.ArrayList[Identifier]()
- store.forEach { (key, _) =>
- if (key._1 == target) ids.add(Identifier.of(key._1.toArray, key._2))
- }
- ids.toArray(new Array[Identifier](0))
- }
-
- override def loadView(ident: Identifier): ViewInfo = {
- val key = (ident.namespace().toSeq, ident.name())
- Option(store.get(key)).getOrElse(throw new NoSuchViewException(ident))
- }
-
- 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())
- if (!store.containsKey(key)) throw new NoSuchViewException(ident)
- store.put(key, info)
- info
- }
-
- override def dropView(ident: Identifier): Boolean = {
- val key = (ident.namespace().toSeq, ident.name())
- store.remove(key) != null
- }
-
- private var catalogName = ""
- override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {
- catalogName = name
- seedDefault()
- }
- override def name(): String = catalogName
-}
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/DataSourceV2MetadataViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala
new file mode 100644
index 000000000000..cc22d01fe079
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetadataViewSuite.scala
@@ -0,0 +1,669 @@
+/*
+ * 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
+
+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, 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 [[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.DataSourceV2MetadataTableSuite]].
+ *
+ * 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._
+
+ override def sparkConf: SparkConf = super.sparkConf
+ .set("spark.sql.catalog.view_catalog", classOf[TestingTableViewCatalog].getName)
+
+ // --- View read path -----------------------------------------------------
+
+ test("read view expands SQL text and applies captured SQL configs") {
+ 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 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))
+ }
+ }
+
+ test("read view resolves unqualified refs via captured current catalog/namespace") {
+ withTable("spark_catalog.default.t") {
+ Seq("a", "b").toDF("col").write.saveAsTable("spark_catalog.default.t")
+ // View text uses the unqualified name `t`; it resolves via the stored
+ // current catalog / namespace properties.
+ checkAnswer(spark.table("view_catalog.ns.test_unqualified_view"), Row("b"))
+ }
+ }
+
+ test("read view resolves unqualified refs via multi-part captured namespace") {
+ // 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 TestingTableViewCatalog resolves to
+ // its own `t` fixture at that namespace.
+ checkAnswer(
+ spark.table("view_catalog.outer_ns.test_unqualified_multi"),
+ Row("multi"))
+ }
+
+ // --- ViewInfo unit tests -----------------------------------------------
+
+ test("multi-part captured namespace round-trips through V1Table.toCatalogTable") {
+ // (a) ViewInfo.Builder stores (cat, Array(db1, db2)) as typed fields.
+ // (b) V1Table.toCatalogTable reads them directly and emits v1's numbered
+ // view.catalogAndNamespace.* keys so (c) the resulting CatalogTable's
+ // `viewCatalogAndNamespace` exposes the full (cat, db1, db2), which is what the v1
+ // view-resolution path consumes to expand unqualified references in the view body.
+ val info = new ViewInfo.Builder()
+ .withSchema(new StructType().add("col", "string"))
+ .withQueryText("SELECT col FROM t")
+ .withCurrentCatalog("my_cat")
+ .withCurrentNamespace(Array("db1", "db2"))
+ .build()
+ 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(
+ catalog, Identifier.of(Array("ns"), "v"), motTable)
+ assert(ct.viewCatalogAndNamespace == Seq("my_cat", "db1", "db2"))
+
+ // Namespace parts containing dots flow through structurally (no string encoding).
+ val infoWeird = new ViewInfo.Builder()
+ .withSchema(new StructType().add("col", "string"))
+ .withQueryText("SELECT col FROM t")
+ .withCurrentCatalog("my_cat")
+ .withCurrentNamespace(Array("weird.db", "normal"))
+ .build()
+ val ctWeird = V1Table.toCatalogTable(
+ catalog, Identifier.of(Array("ns"), "v"), new MetadataTable(infoWeird, "v"))
+ assert(ctWeird.viewCatalogAndNamespace == Seq("my_cat", "weird.db", "normal"))
+ }
+
+ test("view with no captured catalog omits viewCatalogAndNamespace") {
+ val info = new ViewInfo.Builder()
+ .withSchema(new StructType().add("col", "string"))
+ .withQueryText("SELECT * FROM spark_catalog.default.t")
+ .build()
+ 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)
+ }
+
+ // CREATE VIEW behavior tests live in the per-catalog triplet
+ // `sql.execution.command.{,v1/,v2/}.CreateViewSuite{,Base}`.
+
+ // ALTER VIEW behavior tests live in the per-catalog triplet
+ // `sql.execution.command.{,v1/,v2/}.AlterViewAsSuite{,Base}`.
+
+ // --- Pure ViewCatalog (no TableCatalog mixin) ---------------------------
+
+ test("read view from a pure ViewCatalog (no TableCatalog mixin)") {
+ // The analyzer's table-side lookup must skip `loadTable` entirely for catalogs that don't
+ // implement `TableCatalog`; otherwise `asTableCatalog` would throw
+ // MISSING_CATALOG_ABILITY.TABLES and the legitimate `loadView` fallback would never run.
+ withSQLConf(
+ "spark.sql.catalog.view_only" -> classOf[TestingViewOnlyCatalog].getName) {
+ withTable("spark_catalog.default.t") {
+ Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t")
+ // The fixture stores a `pure_v` view whose body filters spark_catalog.default.t.
+ checkAnswer(spark.table("view_only.default.pure_v"), Seq(Row(2), Row(3)))
+ }
+ }
+ }
+
+ test("ALTER VIEW on a pure ViewCatalog (no TableCatalog mixin)") {
+ withSQLConf(
+ "spark.sql.catalog.view_only" -> classOf[TestingViewOnlyCatalog].getName) {
+ val catalog = spark.sessionState.catalogManager.catalog("view_only")
+ .asInstanceOf[TestingViewOnlyCatalog]
+ withTable("spark_catalog.default.t") {
+ Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t")
+ sql("ALTER VIEW view_only.default.pure_v AS " +
+ "SELECT x FROM spark_catalog.default.t WHERE x > 2")
+ assert(catalog.loadView(Identifier.of(Array("default"), "pure_v")).queryText() ==
+ "SELECT x FROM spark_catalog.default.t WHERE x > 2")
+ }
+ }
+ }
+
+ test("cyclic detection distinguishes views across multi-level namespaces") {
+ withTable("spark_catalog.default.t") {
+ 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 -- 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")
+ // Legitimate non-cyclic REPLACE -- new body references a different view that happens to
+ // share the last namespace segment. Must not false-positive.
+ sql("CREATE OR REPLACE VIEW view_catalog.ns1.inner.v AS " +
+ "SELECT x FROM spark_catalog.default.t WHERE x > 1")
+ checkAnswer(spark.table("view_catalog.ns1.inner.v"), Seq(Row(2), Row(3)))
+
+ // Real cycle across the two namespaces must still be caught.
+ val ex = intercept[AnalysisException] {
+ sql("CREATE OR REPLACE VIEW view_catalog.ns1.inner.v AS " +
+ "SELECT x FROM view_catalog.ns2.inner.v")
+ }
+ assert(ex.getCondition == "RECURSIVE_VIEW")
+ }
+ }
+
+ test("view error messages render the full multi-level namespace") {
+ withTable("spark_catalog.default.t") {
+ Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t")
+ 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). 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")
+ }
+ assert(dup.getCondition == "TABLE_OR_VIEW_ALREADY_EXISTS")
+ assert(dup.getMessage.contains("`view_catalog`.`ns1`.`inner`.`v_err`"),
+ s"expected full multi-part name in error, got: ${dup.getMessage}")
+
+ // CREATE OR REPLACE VIEW over a non-view table entry surfaces
+ // `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[TestingTableViewCatalog]
+ val tblIdent = Identifier.of(Array("ns1", "inner"), "t_err")
+ catalog.createTable(
+ tblIdent,
+ new TableInfo.Builder()
+ .withSchema(new StructType().add("col", "string"))
+ .withTableType(TableSummary.EXTERNAL_TABLE_TYPE)
+ .build())
+ try {
+ val notView = intercept[AnalysisException] {
+ sql("CREATE OR REPLACE VIEW view_catalog.ns1.inner.t_err AS " +
+ "SELECT x FROM spark_catalog.default.t")
+ }
+ assert(notView.getCondition == "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE")
+ assert(notView.getMessage.contains("`view_catalog`.`ns1`.`inner`.`t_err`"),
+ s"expected full multi-part name in error, got: ${notView.getMessage}")
+ } finally {
+ catalog.dropTable(tblIdent)
+ }
+
+ // Column-arity mismatch error.
+ val arity = intercept[AnalysisException] {
+ sql("CREATE VIEW view_catalog.ns1.inner.v_arity (a, b) AS " +
+ "SELECT x FROM spark_catalog.default.t")
+ }
+ assert(arity.getMessage.contains("`view_catalog`.`ns1`.`inner`.`v_arity`"),
+ s"expected full multi-part name in error, got: ${arity.getMessage}")
+ }
+ }
+
+ test("ALTER VIEW cyclic detection distinguishes views across multi-level namespaces") {
+ withTable("spark_catalog.default.t") {
+ Seq(1, 2, 3).toDF("x").write.saveAsTable("spark_catalog.default.t")
+
+ sql("CREATE VIEW view_catalog.ns1.inner.v_alter AS " +
+ "SELECT x FROM spark_catalog.default.t")
+ sql("CREATE VIEW view_catalog.ns2.inner.v_alter AS " +
+ "SELECT x FROM view_catalog.ns1.inner.v_alter")
+
+ // 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(
+ spark.table("view_catalog.ns1.inner.v_alter"),
+ Seq(Row(2), Row(3)))
+
+ // Real cycle across the two namespaces must still be caught.
+ val ex = intercept[AnalysisException] {
+ sql("ALTER VIEW view_catalog.ns1.inner.v_alter AS " +
+ "SELECT x FROM view_catalog.ns2.inner.v_alter")
+ }
+ assert(ex.getCondition == "RECURSIVE_VIEW")
+ }
+ }
+
+ test("temp-object reference errors render the full multi-level namespace") {
+ // `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)
+ val ex = intercept[AnalysisException] {
+ sql("CREATE VIEW view_catalog.ns1.inner.v_tempfn AS " +
+ "SELECT temp_udf_multi(x) FROM spark_catalog.default.t")
+ }
+ assert(ex.getCondition == "INVALID_TEMP_OBJ_REFERENCE")
+ assert(ex.getMessage.contains("`view_catalog`.`ns1`.`inner`.`v_tempfn`"),
+ s"expected full multi-part name, got: ${ex.getMessage}")
+ }
+ }
+
+ // --- 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}")
+ }
+
+ // 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).
+ // Without explicit pins they would hit `QueryPlanner`'s `assert(pruned.hasNext, "No plan for
+ // ...")` and surface a raw AssertionError. Pin each to UNSUPPORTED_FEATURE.TABLE_OPERATION.
+
+ test("REFRESH TABLE on a v2 view is rejected") {
+ seedV2View("v_refresh")
+ assertUnsupportedViewOp("REFRESH TABLE view_catalog.default.v_refresh")
+ }
+
+ test("ANALYZE TABLE on a v2 view is rejected") {
+ seedV2View("v_analyze")
+ assertUnsupportedViewOp(
+ "ANALYZE TABLE view_catalog.default.v_analyze COMPUTE STATISTICS")
+ }
+
+ test("ANALYZE TABLE ... FOR COLUMNS on a v2 view is rejected") {
+ seedV2View("v_analyze_cols")
+ assertUnsupportedViewOp(
+ "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") {
+ // `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()
+ 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}`.
+
+ // --- SHOW TABLES / SHOW VIEWS on a v2 catalog --------------------------------
+
+ private def seedV2Table(name: String): Unit = {
+ val catalog = spark.sessionState.catalogManager.catalog("view_catalog")
+ .asInstanceOf[TestingTableViewCatalog]
+ catalog.createTable(
+ Identifier.of(Array("default"), name),
+ new TableInfo.Builder()
+ .withSchema(new StructType().add("x", "int"))
+ .withTableType(TableSummary.EXTERNAL_TABLE_TYPE)
+ .build())
+ }
+
+ 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 missing from SHOW TABLES: $names")
+ rows.foreach(r => assert(!r.getBoolean(2), s"isTemporary must be false: $r"))
+ }
+
+ // SHOW VIEWS behavior tests live in the per-catalog triplet
+ // `sql.execution.command.{,v1/,v2/}.ShowViewsSuite{,Base}`.
+}
+
+/**
+ * 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
+ * [[loadTableOrView]] returns either kind; [[loadTable]] is tables-only per the
+ * [[TableCatalog#loadTable]] contract.
+ */
+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
+ // 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 (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()
+ .withSchema(new StructType().add("col", "string").add("i", "int"))
+ .withQueryText(
+ "SELECT col, col::int AS i FROM spark_catalog.default.t WHERE col = 'b'")
+ .withSqlConfigs(java.util.Collections.singletonMap(
+ SQLConf.ANSI_ENABLED.key, (ident.namespace().head == "ansi").toString))
+ .build())
+ case "test_unqualified_view" =>
+ Some(new ViewInfo.Builder()
+ .withSchema(new StructType().add("col", "string"))
+ .withQueryText("SELECT col FROM t WHERE col = 'b'")
+ .withCurrentCatalog("spark_catalog")
+ .withCurrentNamespace(Array("default"))
+ .build())
+ case "test_unqualified_multi" =>
+ // View whose captured catalog+namespace is view_catalog.ns1.ns2 (two-part). The
+ // unqualified `t` in the body must resolve via that captured context to
+ // view_catalog.ns1.ns2.t, which this catalog also serves (see `t` case below).
+ Some(new ViewInfo.Builder()
+ .withSchema(new StructType().add("col", "string"))
+ .withQueryText("SELECT col FROM t")
+ .withCurrentCatalog("view_catalog")
+ .withCurrentNamespace(Array("ns1", "ns2"))
+ .build())
+ case "t" if ident.namespace().toSeq == Seq("ns1", "ns2") =>
+ // Target of test_unqualified_multi's unqualified reference. Self-contained view so
+ // the test doesn't need external data.
+ Some(new ViewInfo.Builder()
+ .withSchema(new StructType().add("col", "string"))
+ .withQueryText("SELECT 'multi' AS col")
+ .build())
+ case _ => None
+ }
+
+ 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 TableViewCatalog default impls.
+ val key = (ident.namespace().toSeq, ident.name())
+ Option(createdViews.get(key))
+ .orElse(fixtureView(ident))
+ .map(new MetadataTable(_, ident.toString))
+ .getOrElse(throw new NoSuchTableException(ident))
+ }
+
+ override def createTable(ident: Identifier, info: TableInfo): Table = {
+ // Mixed-catalog contract: createTable rejects when a view sits at ident with
+ // TableAlreadyExistsException. The shared `createdViews` keyspace makes `putIfAbsent`
+ // throw uniformly for both table-at-ident and view-at-ident collisions.
+ val key = (ident.namespace().toSeq, ident.name())
+ if (createdViews.putIfAbsent(key, info) != null) {
+ throw new TableAlreadyExistsException(ident)
+ }
+ new MetadataTable(info, ident.toString)
+ }
+
+ /** Test-only accessor: returns the stored TableInfo (table or view) for the identifier. */
+ def getStoredInfo(namespace: Array[String], name: String): TableInfo = {
+ Option(createdViews.get((namespace.toSeq, name))).getOrElse {
+ throw new NoSuchTableException(Identifier.of(namespace, name))
+ }
+ }
+
+ /** Test-only accessor: returns the stored ViewInfo; fails if the entry 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")
+ }
+
+ override def alterTable(ident: Identifier, changes: TableChange*): Table = {
+ throw new RuntimeException("shouldn't be called")
+ }
+ override def dropTable(ident: Identifier): Boolean = {
+ val key = (ident.namespace().toSeq, ident.name())
+ val existing = createdViews.get(key)
+ if (existing == null || existing.isInstanceOf[ViewInfo]) return false
+ createdViews.remove(key) != null
+ }
+ override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = {
+ throw new RuntimeException("shouldn't be called")
+ }
+ override def listTables(namespace: Array[String]): Array[Identifier] = {
+ // Tables only -- views are listed via ViewCatalog.listViews per the new contract.
+ val targetNs = namespace.toSeq
+ val ids = new java.util.ArrayList[Identifier]()
+ createdViews.forEach { (key, info) =>
+ if (key._1 == targetNs && !info.isInstanceOf[ViewInfo]) {
+ ids.add(Identifier.of(key._1.toArray, key._2))
+ }
+ }
+ ids.toArray(new Array[Identifier](0))
+ }
+
+ // ViewCatalog methods. Storage is shared with TableCatalog (mixed-catalog pattern).
+
+ override def listViews(namespace: Array[String]): Array[Identifier] = {
+ val targetNs = namespace.toSeq
+ val ids = new java.util.ArrayList[Identifier]()
+ createdViews.forEach { (key, info) =>
+ if (key._1 == targetNs && 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 (createdViews.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 = createdViews.get(key)
+ if (existing == null || !existing.isInstanceOf[ViewInfo]) {
+ throw new NoSuchViewException(ident)
+ }
+ createdViews.put(key, info)
+ info
+ }
+
+ override def dropView(ident: Identifier): Boolean = {
+ val key = (ident.namespace().toSeq, ident.name())
+ val existing = createdViews.get(key)
+ if (existing == null || !existing.isInstanceOf[ViewInfo]) return false
+ 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
+ }
+ override def name(): String = catalogName
+}
+
+/**
+ * A v2 catalog that does not implement ViewCatalog. Used by capability-gate tests: the gate
+ * fires in `Analyzer.lookupTableOrView(viewOnly=true)` for ALTER VIEW and in
+ * [[CheckViewReferences]] for CREATE VIEW -- in both cases before `loadTable` is called --
+ * so this catalog's content is intentionally empty.
+ */
+class TestingTableOnlyCatalog extends TableCatalog {
+ override def loadTable(ident: Identifier): Table = throw new NoSuchTableException(ident)
+
+ override def alterTable(ident: Identifier, changes: TableChange*): Table =
+ throw new RuntimeException("shouldn't be called")
+ override def dropTable(ident: Identifier): Boolean = false
+ override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit =
+ throw new RuntimeException("shouldn't be called")
+ override def listTables(namespace: Array[String]): Array[Identifier] = Array.empty
+ private var catalogName = ""
+ override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {
+ catalogName = name
+ }
+ override def name(): String = catalogName
+}
+
+/**
+ * A pure [[ViewCatalog]] (no [[TableCatalog]] mixin). Used to exercise that the analyzer's
+ * resolution paths skip the `loadTable` step and fall through to `loadView` for catalogs that
+ * cannot host tables. Pre-seeds a single mutable view at `default.pure_v` so the read and
+ * ALTER VIEW tests can both reach it.
+ */
+class TestingViewOnlyCatalog extends ViewCatalog {
+ private val store =
+ new java.util.concurrent.ConcurrentHashMap[(Seq[String], String), ViewInfo]()
+
+ // Seeded on first `initialize`. Filters `spark_catalog.default.t` so the read test can
+ // assert deterministic output. ALTER VIEW tests overwrite it via `replaceView`.
+ private def seedDefault(): Unit = {
+ val key = (Seq("default"), "pure_v")
+ if (!store.containsKey(key)) {
+ val info = new ViewInfo.Builder()
+ .withSchema(new StructType().add("x", "int"))
+ .withQueryText("SELECT x FROM spark_catalog.default.t WHERE x > 1")
+ .build()
+ store.put(key, info)
+ }
+ }
+
+ override def listViews(namespace: Array[String]): Array[Identifier] = {
+ val target = namespace.toSeq
+ val ids = new java.util.ArrayList[Identifier]()
+ store.forEach { (key, _) =>
+ if (key._1 == target) ids.add(Identifier.of(key._1.toArray, key._2))
+ }
+ ids.toArray(new Array[Identifier](0))
+ }
+
+ override def loadView(ident: Identifier): ViewInfo = {
+ val key = (ident.namespace().toSeq, ident.name())
+ Option(store.get(key)).getOrElse(throw new NoSuchViewException(ident))
+ }
+
+ 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())
+ if (!store.containsKey(key)) throw new NoSuchViewException(ident)
+ store.put(key, info)
+ info
+ }
+
+ override def dropView(ident: Identifier): Boolean = {
+ val key = (ident.namespace().toSeq, ident.name())
+ 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
+ seedDefault()
+ }
+ override def name(): String = catalogName
+}
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..1620425c2abd
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterViewRenameSuiteBase.scala
@@ -0,0 +1,108 @@
+/*
+ * 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}")
+ }
+
+ 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/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