From d275e7ce5a43b8a8e3957a0ca85d63f941172f41 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Tue, 28 Apr 2026 16:06:54 +0000 Subject: [PATCH 01/17] [SPARK-XXXXX][SQL] Support METRIC_VIEW on V2 view catalogs Add support for CREATE VIEW ... WITH METRICS and the corresponding V2 catalog round-trip on top of the ViewCatalog / RelationCatalog APIs introduced by SPARK-52729. The V2 metric-view create path now builds a ViewInfo and calls ViewCatalog.createView. Metric views are distinguished from plain views with PROP_TABLE_TYPE=METRIC_VIEW, which ViewInfo now preserves when callers intentionally set it. This avoids mutating ViewInfo properties after construction while keeping plain views defaulted to VIEW. ViewInfo gains a typed viewDependencies field so catalogs can persist structured table/function dependencies without flattening nested lineage into string properties. The metric-view planner collects direct table dependencies from the analyzed source plan and passes them through this field. Also add first-class CatalogTableType.METRIC_VIEW / TableSummary.METRIC_VIEW_TABLE_TYPE support, V2-to-V1 conversion for MetricView ViewInfo rows, drop-command parity for metric views, metric_view.* descriptive properties, and focused V1/V2 tests for dependency extraction, ViewInfo payloads, DROP VIEW routing, and user-specified column metadata preservation. --- .../sql/connector/catalog/Dependency.java | 39 ++ .../sql/connector/catalog/DependencyList.java | 46 ++ .../connector/catalog/FunctionDependency.java | 38 ++ .../connector/catalog/TableDependency.java | 38 ++ .../sql/connector/catalog/TableSummary.java | 1 + .../spark/sql/connector/catalog/ViewInfo.java | 33 +- .../sql/catalyst/catalog/interface.scala | 11 +- .../spark/sql/connector/catalog/V1Table.scala | 9 +- .../serde/MetricViewCanonical.scala | 33 +- .../metricview/util/MetricViewPlanner.scala | 7 +- .../analysis/ResolveSessionCatalog.scala | 7 +- .../spark/sql/execution/command/ddl.scala | 8 +- .../command/metricViewCommands.scala | 121 ++++- .../spark/sql/execution/command/views.scala | 33 +- .../datasources/v2/DataSourceV2Strategy.scala | 3 +- .../datasources/v2/V2SessionCatalog.scala | 6 +- .../execution/MetricViewV2CatalogSuite.scala | 462 ++++++++++++++++++ .../sql/execution/command/DDLSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 19 files changed, 853 insertions(+), 46 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java new file mode 100644 index 0000000000000..b681b78f1033f --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java @@ -0,0 +1,39 @@ +/* + * 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 org.apache.spark.annotation.Evolving; + +/** + * Represents a dependency of a SQL object such as a view or metric view. + *

+ * A dependency is one of: {@link TableDependency} or {@link FunctionDependency}. + * + * @since 4.2.0 + */ +@Evolving +public interface Dependency { + + static TableDependency table(String tableFullName) { + return new TableDependency(tableFullName); + } + + static FunctionDependency function(String functionFullName) { + return new FunctionDependency(functionFullName); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java new file mode 100644 index 0000000000000..6a1092bd94d73 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java @@ -0,0 +1,46 @@ +/* + * 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.Objects; + +import org.apache.spark.annotation.Evolving; + +/** + * A list of dependencies for a SQL object such as a view or metric view. + *

+ *

+ * + * @param dependencies array of dependencies + * @since 4.2.0 + */ +@Evolving +public record DependencyList(Dependency[] dependencies) { + + public DependencyList { + Objects.requireNonNull(dependencies, "dependencies must not be null"); + } + + public static DependencyList of(Dependency... dependencies) { + return new DependencyList(dependencies); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java new file mode 100644 index 0000000000000..dd76190788c83 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java @@ -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.connector.catalog; + +import java.util.Objects; + +import org.apache.spark.annotation.Evolving; + +/** + * A function dependency of a SQL object. + *

+ * The dependent function is identified by its fully-qualified three-part name + * in the form {@code catalog_name.schema_name.function_name}. + * + * @param functionFullName fully-qualified three-part function name + * @since 4.2.0 + */ +@Evolving +public record FunctionDependency(String functionFullName) implements Dependency { + public FunctionDependency { + Objects.requireNonNull(functionFullName, "functionFullName must not be null"); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java new file mode 100644 index 0000000000000..5e44397139f46 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java @@ -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.connector.catalog; + +import java.util.Objects; + +import org.apache.spark.annotation.Evolving; + +/** + * A table dependency of a SQL object. + *

+ * The dependent table is identified by its fully-qualified three-part name + * in the form {@code catalog_name.schema_name.table_name}. + * + * @param tableFullName fully-qualified three-part table name + * @since 4.2.0 + */ +@Evolving +public record TableDependency(String tableFullName) implements Dependency { + public TableDependency { + Objects.requireNonNull(tableFullName, "tableFullName must not be null"); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java index 8f46a372342a8..17a4f23bdd1f2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java @@ -27,6 +27,7 @@ public interface TableSummary { String EXTERNAL_TABLE_TYPE = "EXTERNAL"; String VIEW_TABLE_TYPE = "VIEW"; String FOREIGN_TABLE_TYPE = "FOREIGN"; + String METRIC_VIEW_TABLE_TYPE = "METRIC_VIEW"; Identifier identifier(); String tableType(); 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 307a5ff486e58..3710dd30b5b97 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 @@ -48,6 +48,7 @@ public class ViewInfo extends TableInfo { private final Map sqlConfigs; private final String schemaMode; private final String[] queryColumnNames; + private final DependencyList viewDependencies; protected ViewInfo(Builder builder) { super(builder); @@ -57,11 +58,14 @@ protected ViewInfo(Builder builder) { this.sqlConfigs = Collections.unmodifiableMap(builder.sqlConfigs); this.schemaMode = builder.schemaMode; this.queryColumnNames = builder.queryColumnNames; - // Force PROP_TABLE_TYPE = VIEW so that `properties()` reflects the typed ViewInfo - // classification. Catalogs and generic viewers reading PROP_TABLE_TYPE from the properties - // bag (e.g. TableCatalog.listTableSummaries default impl, DESCRIBE) see "VIEW" without - // requiring authors to remember to call withTableType(VIEW). - properties().put(TableCatalog.PROP_TABLE_TYPE, TableSummary.VIEW_TABLE_TYPE); + this.viewDependencies = builder.viewDependencies; + // Force PROP_TABLE_TYPE = VIEW by default so that `properties()` reflects the typed + // ViewInfo classification. Metric views intentionally set PROP_TABLE_TYPE = METRIC_VIEW + // before construction and must retain that more-specific view kind. + String tableType = properties().get(TableCatalog.PROP_TABLE_TYPE); + if (!TableSummary.METRIC_VIEW_TABLE_TYPE.equals(tableType)) { + properties().put(TableCatalog.PROP_TABLE_TYPE, TableSummary.VIEW_TABLE_TYPE); + } } /** The SQL text of the view. */ @@ -102,6 +106,14 @@ protected ViewInfo(Builder builder) { */ public String[] queryColumnNames() { return queryColumnNames; } + /** + * Returns the structured list of objects this view depends on (source tables and functions), + * or {@code null} if no dependency list was supplied. Unlike other view metadata which is + * encoded into {@link #properties()}, dependency lists are a first-class field because their + * nested structure does not round-trip cleanly through flat string properties. + */ + public DependencyList viewDependencies() { return viewDependencies; } + public static class Builder extends BaseBuilder { private String queryText; private String currentCatalog; @@ -109,6 +121,7 @@ public static class Builder extends BaseBuilder { private Map sqlConfigs = new HashMap<>(); private String schemaMode; private String[] queryColumnNames = new String[0]; + private DependencyList viewDependencies = null; @Override protected Builder self() { return this; } @@ -143,6 +156,16 @@ public Builder withQueryColumnNames(String[] queryColumnNames) { return this; } + /** + * Sets the structured dependency list for this view. Source tables and functions referenced + * by the view text should be recorded here so downstream consumers (e.g. catalogs persisting + * lineage) can access them without re-analyzing the view body. + */ + public Builder withViewDependencies(DependencyList viewDependencies) { + this.viewDependencies = viewDependencies; + return this; + } + @Override public ViewInfo build() { Objects.requireNonNull(columns, "columns should not be null"); 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 1c4362bfd3ed7..ed134589c2540 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 @@ -742,15 +742,11 @@ object CatalogTable { val VIEW_CATALOG_AND_NAMESPACE = VIEW_PREFIX + "catalogAndNamespace.numParts" val VIEW_CATALOG_AND_NAMESPACE_PART_PREFIX = VIEW_PREFIX + "catalogAndNamespace.part." - // Property to indicate that a VIEW is actually a METRIC VIEW - val VIEW_WITH_METRICS = VIEW_PREFIX + "viewWithMetrics" - /** - * Check if a CatalogTable is a metric view by looking at its properties. + * Check if a CatalogTable is a metric view. */ def isMetricView(table: CatalogTable): Boolean = { - table.tableType == CatalogTableType.VIEW && - table.properties.get(VIEW_WITH_METRICS).contains("true") + table.tableType == CatalogTableType.METRIC_VIEW } // Convert the current catalog and namespace to properties. @@ -1089,8 +1085,9 @@ object CatalogTableType { val EXTERNAL = new CatalogTableType("EXTERNAL") val MANAGED = new CatalogTableType("MANAGED") val VIEW = new CatalogTableType("VIEW") + val METRIC_VIEW = new CatalogTableType("METRIC_VIEW") - val tableTypes = Seq(EXTERNAL, MANAGED, VIEW) + val tableTypes = Seq(EXTERNAL, MANAGED, VIEW, METRIC_VIEW) } 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 a1fb2c1c84e40..8a47cac8e7962 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 @@ -105,6 +105,7 @@ private[sql] object V1Table { case CatalogTableType.EXTERNAL => Some(TableSummary.EXTERNAL_TABLE_TYPE) case CatalogTableType.MANAGED => Some(TableSummary.MANAGED_TABLE_TYPE) case CatalogTableType.VIEW => Some(TableSummary.VIEW_TABLE_TYPE) + case CatalogTableType.METRIC_VIEW => Some(TableSummary.METRIC_VIEW_TABLE_TYPE) case _ => None } } @@ -195,9 +196,15 @@ private[sql] object V1Table { val schemaModeProps = Option(info.schemaMode) .map(m => Map(CatalogTable.VIEW_SCHEMA_MODE -> m)) .getOrElse(Map.empty) + // ViewInfo always represents a view-like table, but PROP_TABLE_TYPE may further refine the + // kind (e.g. METRIC_VIEW). Default to plain VIEW when no refinement is supplied. + val tableType = props.get(TableCatalog.PROP_TABLE_TYPE) match { + case Some(TableSummary.METRIC_VIEW_TABLE_TYPE) => CatalogTableType.METRIC_VIEW + case _ => CatalogTableType.VIEW + } CatalogTable( identifier = ident.asLegacyTableIdentifier(catalog.name()), - tableType = CatalogTableType.VIEW, + tableType = tableType, storage = CatalogStorageFormat.empty, schema = CatalogV2Util.v2ColumnsToStructType(info.columns), owner = props.getOrElse(TableCatalog.PROP_OWNER, ""), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala index 2e76a13741d09..2a6f4d7c57c5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala @@ -94,7 +94,7 @@ private[sql] object Source { if (sourceText.isEmpty) { throw MetricViewValidationException("Source cannot be empty") } - Try(CatalystSqlParser.parseTableIdentifier(sourceText)) match { + Try(CatalystSqlParser.parseMultipartIdentifier(sourceText)) match { case Success(_) => AssetSource(sourceText) case Failure(_) => Try(CatalystSqlParser.parseQuery(sourceText)) match { @@ -167,4 +167,33 @@ private[sql] case class MetricView( version: String, from: Source, where: Option[String] = None, - select: Seq[Column]) + select: Seq[Column]) { + + /** + * Returns a set of table properties describing this metric view's source and + * filter clauses. Mirrors the property keys used by the canonical metric view + * representation on other Spark platforms so consumers of the catalog see a + * consistent property layout. + */ + def getProperties: Map[String, String] = { + val base = Map(MetricView.PROP_FROM_TYPE -> from.sourceType.toString) + val fromProps = from match { + case asset: AssetSource => + base + (MetricView.PROP_FROM_NAME -> asset.name) + case sql: SQLSource => + base + (MetricView.PROP_FROM_SQL -> MetricView.truncate(sql.sql)) + } + where.fold(fromProps)(w => + fromProps + (MetricView.PROP_WHERE -> MetricView.truncate(w))) + } +} + +private[sql] object MetricView { + final val PROP_FROM_TYPE = "metric_view.from.type" + final val PROP_FROM_NAME = "metric_view.from.name" + final val PROP_FROM_SQL = "metric_view.from.sql" + final val PROP_WHERE = "metric_view.where" + + private def truncate(value: String): String = + value.take(Constants.MAXIMUM_PROPERTY_SIZE) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala index 121d908eda90b..b2d39057750ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.metricview.logical.MetricViewPlaceholder import org.apache.spark.sql.metricview.serde.{AssetSource, MetricView, MetricViewFactory, MetricViewValidationException, MetricViewYAMLParsingException, SQLSource} import org.apache.spark.sql.types.StructType @@ -65,9 +64,11 @@ object MetricViewPlanner { MetricViewFactory.fromYAML(yaml) } catch { case e: MetricViewValidationException => - throw QueryCompilationErrors.invalidLiteralForWindowDurationError() + throw SparkException.internalError( + s"Invalid metric view YAML: ${e.getMessage}", e) case e: MetricViewYAMLParsingException => - throw QueryCompilationErrors.invalidLiteralForWindowDurationError() + throw SparkException.internalError( + s"Failed to parse metric view YAML: ${e.getMessage}", e) } val source = metricView.from match { case asset: AssetSource => UnresolvedRelation(sqlParser.parseMultipartIdentifier(asset.name)) 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 29bcb98f3f099..8f0b664e10c5b 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 @@ -342,9 +342,10 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) DropTableCommand(ident, ifExists, isView = true, purge = false) // ViewCatalog catalogs fall through to `DataSourceV2Strategy`, which routes DROP VIEW to - // `ViewCatalog.dropView`. Other non-session catalogs get `MISSING_CATALOG_ABILITY.VIEWS`, - // matching the error raised from `CheckViewReferences` for CREATE/ALTER VIEW and from the - // analyzer gate on UnresolvedView. + // `ViewCatalog.dropView` (this also covers METRIC_VIEW since metric views are persisted + // through the same ViewCatalog interface). Other non-session catalogs get + // `MISSING_CATALOG_ABILITY.VIEWS`, matching the error raised from `CheckViewReferences` for + // CREATE/ALTER VIEW and from the analyzer gate on UnresolvedView. case DropView(r @ ResolvedIdentifier(catalog, ident), ifExists) if !catalog.isInstanceOf[ViewCatalog] => if (catalog == FakeSystemCatalog) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 160b007b547f6..cd8ac53a58443 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -232,7 +232,8 @@ case class DropTableCommand( // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view // issue an exception. catalog.getTableMetadata(tableName).tableType match { - case CatalogTableType.VIEW if !isView => + // Both VIEW and METRIC_VIEW are conceptually views and must be dropped via DROP VIEW. + case CatalogTableType.VIEW | CatalogTableType.METRIC_VIEW if !isView => throw QueryCompilationErrors.wrongCommandForObjectTypeError( operation = "DROP TABLE", requiredType = s"${CatalogTableType.EXTERNAL.name} or ${CatalogTableType.MANAGED.name}", @@ -240,10 +241,11 @@ case class DropTableCommand( foundType = catalog.getTableMetadata(tableName).tableType.name, alternative = "DROP VIEW" ) - case o if o != CatalogTableType.VIEW && isView => + case o if o != CatalogTableType.VIEW && o != CatalogTableType.METRIC_VIEW && isView => throw QueryCompilationErrors.wrongCommandForObjectTypeError( operation = "DROP VIEW", - requiredType = CatalogTableType.VIEW.name, + requiredType = + s"${CatalogTableType.VIEW.name} or ${CatalogTableType.METRIC_VIEW.name}", objectName = catalog.getTableMetadata(tableName).qualifiedName, foundType = o.name, alternative = "DROP TABLE" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala index 623685f6c20a7..6f16fb6041f77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala @@ -17,13 +17,20 @@ package org.apache.spark.sql.execution.command +import scala.jdk.CollectionConverters._ + import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{QueryPlanningTracker, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, SchemaUnsupported} -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HiveTableRelation} +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, View} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Dependency, DependencyList, TableCatalog, TableSummary, ViewCatalog, ViewInfo} import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.metricview.serde.MetricViewFactory import org.apache.spark.sql.metricview.util.MetricViewPlanner import org.apache.spark.sql.types.StructType @@ -39,15 +46,17 @@ case class CreateMetricViewCommand( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - val name = child match { + child match { + case v: ResolvedIdentifier if !CatalogV2Util.isSessionCatalog(v.catalog) => + createMetricViewInV2Catalog(sparkSession, v) case v: ResolvedIdentifier => - v.identifier.asTableIdentifier + createMetricViewInSessionCatalog(sparkSession, v) case _ => throw SparkException.internalError( s"Failed to resolve identifier for creating metric view") } - val analyzed = MetricViewHelper.analyzeMetricViewText(sparkSession, name, originalText) + } + private def validateUserColumns(name: TableIdentifier, analyzed: LogicalPlan): Unit = { if (userSpecifiedColumns.nonEmpty) { if (userSpecifiedColumns.length > analyzed.output.length) { throw QueryCompilationErrors.cannotCreateViewNotEnoughColumnsError( @@ -57,6 +66,15 @@ case class CreateMetricViewCommand( name.nameParts, userSpecifiedColumns.map(_._1), analyzed) } } + } + + private def createMetricViewInSessionCatalog( + sparkSession: SparkSession, + resolved: ResolvedIdentifier): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + val name = resolved.identifier.asTableIdentifier + val analyzed = MetricViewHelper.analyzeMetricViewText(sparkSession, name, originalText) + validateUserColumns(name, analyzed) catalog.createTable( ViewHelper.prepareTable( sparkSession, name, Some(originalText), analyzed, userSpecifiedColumns, @@ -65,6 +83,66 @@ case class CreateMetricViewCommand( ignoreIfExists = allowExisting) Seq.empty } + + private def createMetricViewInV2Catalog( + sparkSession: SparkSession, + resolved: ResolvedIdentifier): Seq[Row] = { + // Metric views are persisted through the same `ViewCatalog` interface as plain views; the + // only differences are `PROP_TABLE_TYPE = METRIC_VIEW` (so `V1Table.toCatalogTable` maps the + // round-tripped row back to `CatalogTableType.METRIC_VIEW`), the `metric_view.*` descriptor + // properties produced by `MetricView.getProperties`, and the typed `viewDependencies` field. + val viewCatalog = resolved.catalog match { + case vc: ViewCatalog => vc + case other => + throw QueryCompilationErrors.missingCatalogViewsAbilityError(other) + } + val ident = resolved.identifier + val name = ident.asTableIdentifier + + val analyzed = MetricViewHelper.analyzeMetricViewText(sparkSession, name, originalText) + validateUserColumns(name, analyzed) + + // `retainMetadata = true` preserves the per-column `metric_view.type` / `metric_view.expr` + // metadata that `ResolveMetricView.buildMetricViewOutput` attaches, even when the user + // supplies column names with comments (same contract as the V1 session-catalog path, which + // goes through `ViewHelper.prepareTable` with `isMetricView = true`). + val aliasedSchema = ViewHelper + .aliasPlan(sparkSession, analyzed, userSpecifiedColumns, retainMetadata = true) + .schema + + // Describe this metric view's source and filter as user-visible properties so catalogs and + // tooling can inspect them without re-parsing the YAML body. + val metricView = MetricViewFactory.fromYAML(originalText) + val viewProperties = new java.util.HashMap[String, String]() + properties.foreach { case (k, v) => viewProperties.put(k, v) } + metricView.getProperties.foreach { case (k, v) => viewProperties.put(k, v) } + viewProperties.put(TableCatalog.PROP_TABLE_TYPE, TableSummary.METRIC_VIEW_TABLE_TYPE) + + val sourceTableNames = MetricViewHelper.collectTableDependencies(analyzed) + val deps = if (sourceTableNames.nonEmpty) { + DependencyList.of(sourceTableNames.map(Dependency.table): _*) + } else { + null + } + + val manager = sparkSession.sessionState.catalogManager + val builder = new ViewInfo.Builder() + .withSchema(aliasedSchema) + .withProperties(viewProperties) + .withQueryText(originalText) + .withCurrentCatalog(manager.currentCatalog.name) + .withCurrentNamespace(manager.currentNamespace) + .withSqlConfigs( + ViewHelper.sqlConfigsToProps(sparkSession.sessionState.conf, "").asJava) + .withSchemaMode(SchemaUnsupported.toString) + .withQueryColumnNames(analyzed.output.map(_.name).toArray) + .withViewDependencies(deps) + comment.foreach(builder.withComment) + + viewCatalog.createView(ident, builder.build()) + Seq.empty + } + override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = { copy(child = newChild) } @@ -73,6 +151,37 @@ case class CreateMetricViewCommand( case class AlterMetricViewCommand(child: LogicalPlan, originalText: String) object MetricViewHelper { + + /** + * Walks the analyzed plan to collect direct table/view dependencies. + * Stops recursion at relation leaf nodes and persistent View nodes so that only + * direct (not transitive) dependencies are recorded. + */ + private[execution] def collectTableDependencies(plan: LogicalPlan): Seq[String] = { + val tables = scala.collection.mutable.ArrayBuffer.empty[String] + def traverse(p: LogicalPlan): Unit = p match { + case v: View if !v.isTempView => + tables += v.desc.identifier.unquotedString + case r: DataSourceV2Relation if r.catalog.isDefined && r.identifier.isDefined => + val cat = r.catalog.get.name() + val ns = r.identifier.get.namespace().mkString(".") + val name = r.identifier.get.name() + tables += s"$cat.$ns.$name" + case r: HiveTableRelation => + tables += r.tableMeta.identifier.unquotedString + case r: LogicalRelation if r.catalogTable.isDefined => + tables += r.catalogTable.get.identifier.unquotedString + case other => + other.children.foreach(traverse) + other.expressions.foreach(_.foreach { + case s: SubqueryExpression => traverse(s.plan) + case _ => + }) + } + traverse(plan) + tables.distinct.toSeq + } + def analyzeMetricViewText( session: SparkSession, name: TableIdentifier, 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 807342c0e90c2..098cc35a63f83 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 @@ -866,24 +866,23 @@ object ViewHelper extends SQLConfHelper with Logging with CapturesConfig { if (originalText.isEmpty) { throw QueryCompilationErrors.createPersistedViewFromDatasetAPINotAllowedError() } + // For metric views, preserve the per-column metadata (`metric_view.type` / `metric_view.expr`) + // that the analyzer attaches to each dimension/measure `Alias`, even when the user supplies + // column names with comments. val aliasedSchema = CharVarcharUtils.getRawSchema( - aliasPlan(session, analyzedPlan, userSpecifiedColumns).schema, session.sessionState.conf) + aliasPlan(session, analyzedPlan, userSpecifiedColumns, retainMetadata = isMetricView).schema, + session.sessionState.conf) val newProperties = generateViewProperties( properties, session, analyzedPlan.schema.fieldNames, aliasedSchema.fieldNames, viewSchemaMode) - // Add property to indicate if this is a metric view - val finalProperties = if (isMetricView) { - newProperties + (CatalogTable.VIEW_WITH_METRICS -> "true") - } else { - newProperties - } + val tableType = if (isMetricView) CatalogTableType.METRIC_VIEW else CatalogTableType.VIEW CatalogTable( identifier = name, - tableType = CatalogTableType.VIEW, + tableType = tableType, storage = CatalogStorageFormat.empty, schema = aliasedSchema, - properties = finalProperties, + properties = newProperties, viewOriginalText = originalText, viewText = originalText, comment = comment, @@ -894,18 +893,30 @@ object ViewHelper extends SQLConfHelper with Logging with CapturesConfig { /** * If `userSpecifiedColumns` is defined, alias the analyzed plan to the user specified columns, * else return the analyzed plan directly. + * + * When `retainMetadata` is true, any existing column metadata on the analyzed attribute + * (for example the `metric_view.type` / `metric_view.expr` keys the analyzer attaches to + * metric-view columns) is preserved in the re-aliased projection. The no-comment branch + * already preserves `attr.metadata` transitively via `child.metadata` on the new `Alias`; + * the comment branch needs an explicit merge because it sets `explicitMetadata` to a + * freshly constructed metadata object. */ def aliasPlan( session: SparkSession, analyzedPlan: LogicalPlan, - userSpecifiedColumns: Seq[(String, Option[String])]): LogicalPlan = { + userSpecifiedColumns: Seq[(String, Option[String])], + retainMetadata: Boolean = false): LogicalPlan = { if (userSpecifiedColumns.isEmpty) { analyzedPlan } else { val projectList = analyzedPlan.output.zip(userSpecifiedColumns).map { case (attr, (colName, None)) => Alias(attr, colName)() case (attr, (colName, Some(colComment))) => - val meta = new MetadataBuilder().putString("comment", colComment).build() + val builder = new MetadataBuilder() + if (retainMetadata) { + builder.withMetadata(attr.metadata) + } + val meta = builder.putString("comment", colComment).build() Alias(attr, colName)(explicitMetadata = Some(meta)) } session.sessionState.executePlan(Project(projectList, analyzedPlan)).analyzed 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 d9e915f82e07f..eacb97a069f65 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 @@ -563,8 +563,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } case DropTable(r: ResolvedIdentifier, ifExists, purge) => + val tableCatalog = r.catalog.asTableCatalog val invalidateFunc = () => CommandUtils.uncacheTableOrView(session, r) - DropTableExec(r.catalog.asTableCatalog, r.identifier, ifExists, purge, invalidateFunc) :: Nil + DropTableExec(tableCatalog, r.identifier, ifExists, purge, invalidateFunc) :: Nil case _: NoopCommand => LocalTableScanExec(Nil, Nil, None) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index d21b5c730f0ca..0d35380c61fdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -334,8 +334,10 @@ class V2SessionCatalog(catalog: SessionCatalog) private def dropTableInternal(ident: Identifier, purge: Boolean = false): Boolean = { try { loadTable(ident) match { - case V1Table(v1Table) if v1Table.tableType == CatalogTableType.VIEW && - !SQLConf.get.getConf(SQLConf.DROP_TABLE_VIEW_ENABLED) => + case V1Table(v1Table) + if (v1Table.tableType == CatalogTableType.VIEW || + v1Table.tableType == CatalogTableType.METRIC_VIEW) && + !SQLConf.get.getConf(SQLConf.DROP_TABLE_VIEW_ENABLED) => throw QueryCompilationErrors.wrongCommandForObjectTypeError( operation = "DROP TABLE", requiredType = s"${CatalogTableType.EXTERNAL.name} or" + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala new file mode 100644 index 0000000000000..18acc82957257 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -0,0 +1,462 @@ +/* + * 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 + +import java.util.concurrent.ConcurrentHashMap + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.analysis.{NoSuchViewException, ViewAlreadyExistsException} +import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog, MetadataOnlyTable, RelationCatalog, Table, TableCatalog, TableDependency, TableSummary, ViewInfo} +import org.apache.spark.sql.metricview.serde.{AssetSource, Column, Constants, DimensionExpression, MeasureExpression, MetricView, MetricViewFactory, SQLSource} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.Metadata + +/** + * Tests that exercise [[org.apache.spark.sql.execution.command.CreateMetricViewCommand]] on a + * non-session V2 catalog. Metric views are persisted through the same [[ViewCatalog]] interface + * as plain views; the only marker that distinguishes them is `PROP_TABLE_TYPE = METRIC_VIEW` + * plus the typed `viewDependencies` field on [[ViewInfo]]. The recording catalog used here is a + * minimal [[RelationCatalog]] so the same instance can also host the source table referenced by + * the metric view's YAML. + */ +class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { + + import testImplicits._ + + private val testCatalogName = "testcat" + private val testNamespace = "ns" + private val sourceTableName = "events" + private val fullSourceTableName = + s"$testCatalogName.$testNamespace.$sourceTableName" + private val metricViewName = "mv" + private val fullMetricViewName = + s"$testCatalogName.$testNamespace.$metricViewName" + + private val metricViewColumns = Seq( + Column("region", DimensionExpression("region"), 0), + Column("count_sum", MeasureExpression("sum(count)"), 1)) + + private val testTableData = Seq( + ("region_1", 1, 5.0), + ("region_2", 2, 10.0)) + + override protected def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set( + s"spark.sql.catalog.$testCatalogName", + classOf[MetricViewRecordingCatalog].getName) + // A catalog that does not implement ViewCatalog - used for the negative gate test. + spark.conf.set( + s"spark.sql.catalog.${MetricViewV2CatalogSuite.noViewCatalogName}", + classOf[InMemoryTableCatalog].getName) + } + + override protected def afterAll(): Unit = { + spark.conf.unset(s"spark.sql.catalog.$testCatalogName") + spark.conf.unset( + s"spark.sql.catalog.${MetricViewV2CatalogSuite.noViewCatalogName}") + super.afterAll() + } + + private def withTestCatalogTables(body: => Unit): Unit = { + MetricViewRecordingCatalog.reset() + testTableData.toDF("region", "count", "price") + .createOrReplaceTempView("metric_view_v2_source") + try { + sql( + s"""CREATE TABLE $fullSourceTableName + |USING foo AS SELECT * FROM metric_view_v2_source""".stripMargin) + body + } finally { + sql(s"DROP VIEW IF EXISTS $fullMetricViewName") + sql(s"DROP TABLE IF EXISTS $fullSourceTableName") + spark.catalog.dropTempView("metric_view_v2_source") + MetricViewRecordingCatalog.reset() + } + } + + private def createMetricView( + name: String, + metricView: MetricView, + comment: Option[String] = None): String = { + val yaml = MetricViewFactory.toYAML(metricView) + val commentClause = comment.map(c => s"\nCOMMENT '$c'").getOrElse("") + sql( + s"""CREATE VIEW $name + |WITH METRICS$commentClause + |LANGUAGE YAML + |AS + |$$$$ + |$yaml + |$$$$""".stripMargin) + yaml + } + + private def capturedViewInfo(): ViewInfo = { + val ident = Identifier.of(Array(testNamespace), metricViewName) + val info = MetricViewRecordingCatalog.capturedViews.get(ident) + assert(info != null, + s"Expected ViewInfo for $ident to be captured by the V2 catalog") + info + } + + test("V2 catalog receives METRIC_VIEW table type and view text via ViewInfo") { + withTestCatalogTables { + val metricView = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + val yaml = createMetricView(fullMetricViewName, metricView) + + val info = capturedViewInfo() + // PROP_TABLE_TYPE is overwritten to METRIC_VIEW after `ViewInfo`'s constructor stamps it + // to VIEW; this is the marker `V1Table.toCatalogTable` reads to map the round-tripped row + // back to `CatalogTableType.METRIC_VIEW`. + assert(info.properties().get(TableCatalog.PROP_TABLE_TYPE) + === TableSummary.METRIC_VIEW_TABLE_TYPE) + assert(info.queryText() === yaml) + + val deps = info.viewDependencies() + assert(deps != null) + assert(deps.dependencies().length === 1) + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.tableFullName() === fullSourceTableName) + } + } + + test("V2 catalog path populates metric_view.* + view context + sql configs on ViewInfo") { + withTestCatalogTables { + val metricView = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = Some("count > 0"), + select = metricViewColumns) + createMetricView(fullMetricViewName, metricView) + + val info = capturedViewInfo() + val props = info.properties() + + // metric_view.* descriptive properties (mirrors DBR SingleSourceMetricView). + assert(props.get(MetricView.PROP_FROM_TYPE) === "ASSET") + assert(props.get(MetricView.PROP_FROM_NAME) === fullSourceTableName) + assert(props.get(MetricView.PROP_FROM_SQL) === null) + assert(props.get(MetricView.PROP_WHERE) === "count > 0") + + // SQL configs and current catalog/namespace are first-class typed fields on ViewInfo, no + // longer encoded into properties for V2 catalogs. + assert(info.sqlConfigs().size > 0, + s"Expected at least one captured SQL config; got ${info.sqlConfigs()}") + assert(info.currentCatalog() === + spark.sessionState.catalogManager.currentCatalog.name()) + assert(info.currentNamespace().toSeq === + spark.sessionState.catalogManager.currentNamespace.toSeq) + } + } + + test("DROP VIEW succeeds on a V2 metric view") { + withTestCatalogTables { + val metricView = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, metricView) + val ident = Identifier.of(Array(testNamespace), metricViewName) + + assert(MetricViewRecordingCatalog.capturedViews.containsKey(ident)) + + sql(s"DROP VIEW $fullMetricViewName") + assert(!MetricViewRecordingCatalog.capturedViews.containsKey(ident)) + } + } + + test("DROP VIEW IF EXISTS on a non-existent V2 metric view is a no-op") { + withTestCatalogTables { + sql(s"DROP VIEW IF EXISTS $testCatalogName.$testNamespace.does_not_exist") + } + } + + test("V2 catalog path captures SQL source and comment") { + withTestCatalogTables { + val metricView = MetricView( + "0.1", + SQLSource(s"SELECT * FROM $fullSourceTableName"), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, metricView, comment = Some("my mv")) + + val info = capturedViewInfo() + val props = info.properties() + assert(props.get(TableCatalog.PROP_TABLE_TYPE) + === TableSummary.METRIC_VIEW_TABLE_TYPE) + assert(props.get(MetricView.PROP_FROM_TYPE) === "SQL") + assert(props.get(MetricView.PROP_FROM_NAME) === null) + assert(props.get(MetricView.PROP_FROM_SQL) === + s"SELECT * FROM $fullSourceTableName") + assert(props.get(TableCatalog.PROP_COMMENT) === "my mv") + + val deps = info.viewDependencies() + assert(deps != null && deps.dependencies().length === 1) + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.tableFullName() === fullSourceTableName) + } + } + + test("metric view columns carry metric_view.type / metric_view.expr in column metadata") { + withTestCatalogTables { + val metricView = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, metricView) + + val cols = capturedViewInfo().columns() + assert(cols.length === metricViewColumns.length) + + val byName = cols.map(c => c.name() -> c).toMap + def metadataOf(name: String): Metadata = + Metadata.fromJson(Option(byName(name).metadataInJSON()).getOrElse("{}")) + + val regionMeta = metadataOf("region") + assert(regionMeta.getString(Constants.COLUMN_TYPE_PROPERTY_KEY) === "dimension") + assert(regionMeta.getString(Constants.COLUMN_EXPR_PROPERTY_KEY) === "region") + + val countMeta = metadataOf("count_sum") + assert(countMeta.getString(Constants.COLUMN_TYPE_PROPERTY_KEY) === "measure") + assert(countMeta.getString(Constants.COLUMN_EXPR_PROPERTY_KEY) === "sum(count)") + } + } + + test("user-specified column names with comments preserve metric_view.* metadata") { + withTestCatalogTables { + val metricView = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + val yaml = MetricViewFactory.toYAML(metricView) + // Give both columns new names, and a comment on each. Without the `retainMetadata` + // fix to `ViewHelper.aliasPlan`, the metric_view.* keys disappear here. + sql( + s"""CREATE VIEW $fullMetricViewName (reg COMMENT 'region alias', n COMMENT 'count') + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |$yaml + |$$$$""".stripMargin) + + val cols = capturedViewInfo().columns() + val byName = cols.map(c => c.name() -> c).toMap + assert(byName.keySet === Set("reg", "n")) + + def metadataOf(name: String): Metadata = + Metadata.fromJson(Option(byName(name).metadataInJSON()).getOrElse("{}")) + + val regMeta = metadataOf("reg") + assert(regMeta.getString(Constants.COLUMN_TYPE_PROPERTY_KEY) === "dimension") + assert(regMeta.getString(Constants.COLUMN_EXPR_PROPERTY_KEY) === "region") + // `CatalogV2Util.structTypeToV2Columns` peels "comment" off into `Column.comment()` + // rather than leaving it inside `metadataInJSON`; assert via the V2 column accessor. + assert(byName("reg").comment() === "region alias") + + val nMeta = metadataOf("n") + assert(nMeta.getString(Constants.COLUMN_TYPE_PROPERTY_KEY) === "measure") + assert(nMeta.getString(Constants.COLUMN_EXPR_PROPERTY_KEY) === "sum(count)") + assert(byName("n").comment() === "count") + } + } + + test("dependency extraction: SQL source JOIN captures both tables") { + withTestCatalogTables { + val secondSource = s"$testCatalogName.$testNamespace.customers" + sql( + s"""CREATE TABLE $secondSource (id INT, name STRING) + |USING foo""".stripMargin) + try { + val joinSql = + s"SELECT c.name, t.count FROM $fullSourceTableName t " + + s"JOIN $secondSource c ON t.count = c.id" + val metricView = MetricView( + "0.1", + SQLSource(joinSql), + where = None, + select = Seq( + Column("name", DimensionExpression("name"), 0), + Column("count_sum", MeasureExpression("sum(count)"), 1))) + createMetricView(fullMetricViewName, metricView) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null) + val depNames = + deps.dependencies().map(_.asInstanceOf[TableDependency].tableFullName()).toSet + assert(depNames === Set(fullSourceTableName, secondSource), + s"Expected dependencies on both source tables, got $depNames") + } finally { + sql(s"DROP TABLE IF EXISTS $secondSource") + } + } + } + + test("dependency extraction: SQL source subquery deduplicates same-table references") { + withTestCatalogTables { + val subquerySql = + s"SELECT * FROM $fullSourceTableName " + + s"WHERE count > (SELECT avg(count) FROM $fullSourceTableName)" + val metricView = MetricView( + "0.1", + SQLSource(subquerySql), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, metricView) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null && deps.dependencies().length === 1, + s"Expected 1 deduplicated dependency, got " + + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.tableFullName() === fullSourceTableName) + } + } + + test("dependency extraction: SQL source self-join deduplicates same-table references") { + withTestCatalogTables { + val selfJoinSql = + s"SELECT a.region AS a_region, a.count AS a_count " + + s"FROM $fullSourceTableName a JOIN $fullSourceTableName b " + + s"ON a.region = b.region" + val metricView = MetricView( + "0.1", + SQLSource(selfJoinSql), + where = None, + select = Seq( + Column("region", DimensionExpression("a_region"), 0), + Column("count_sum", MeasureExpression("sum(a_count)"), 1))) + createMetricView(fullMetricViewName, metricView) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null && deps.dependencies().length === 1, + s"Expected 1 deduplicated dependency for self-join, got " + + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.tableFullName() === fullSourceTableName) + } + } + + test("CREATE VIEW ... WITH METRICS on a non-ViewCatalog catalog fails with " + + "MISSING_CATALOG_ABILITY.VIEWS") { + val ex = intercept[AnalysisException] { + sql( + s"""CREATE VIEW ${MetricViewV2CatalogSuite.noViewCatalogName}.default.mv + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |${MetricViewFactory.toYAML(MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns))} + |$$$$""".stripMargin) + } + assert(ex.getCondition === "MISSING_CATALOG_ABILITY") + assert(ex.getMessage.contains("VIEWS")) + } +} + +object MetricViewV2CatalogSuite { + val noViewCatalogName: String = "testcat_no_view" +} + +/** + * Minimal [[RelationCatalog]] used by [[MetricViewV2CatalogSuite]]. Layers `ViewCatalog` + * methods over [[InMemoryTableCatalog]] (which provides table storage + namespace ops) and + * captures every [[ViewInfo]] passed to `createView` so tests can inspect the typed payload. + * + * The metric-view CREATE path goes via `ViewCatalog.createView`, so the captured map keys are + * the view identifiers; the source table created by the test fixture is stored separately in + * the inherited table catalog. + */ +class MetricViewRecordingCatalog extends InMemoryTableCatalog with RelationCatalog { + private val views = + new ConcurrentHashMap[(Seq[String], String), ViewInfo]() + + // -- ViewCatalog methods -- + + override def listViews(namespace: Array[String]): Array[Identifier] = { + val target = namespace.toSeq + val out = new java.util.ArrayList[Identifier]() + views.forEach { (key, _) => + if (key._1 == target) out.add(Identifier.of(key._1.toArray, key._2)) + } + out.asScala.toArray + } + + // `loadView`, `tableExists`, and `viewExists` are inherited from `RelationCatalog`'s + // defaults, which derive from `loadRelation` -- a stored `ViewInfo` is wrapped in + // `MetadataOnlyTable` by `loadRelation` and the defaults unwrap it correctly. + + override def createView(ident: Identifier, info: ViewInfo): ViewInfo = { + val key = (ident.namespace().toSeq, ident.name()) + if (views.putIfAbsent(key, info) != null) { + throw new ViewAlreadyExistsException(ident) + } + MetricViewRecordingCatalog.capturedViews.put(ident, info) + info + } + + override def replaceView(ident: Identifier, info: ViewInfo): ViewInfo = { + val key = (ident.namespace().toSeq, ident.name()) + if (!views.containsKey(key)) throw new NoSuchViewException(ident) + views.put(key, info) + MetricViewRecordingCatalog.capturedViews.put(ident, info) + info + } + + override def dropView(ident: Identifier): Boolean = { + val key = (ident.namespace().toSeq, ident.name()) + val removed = views.remove(key) != null + if (removed) { + MetricViewRecordingCatalog.capturedViews.remove(ident) + } + removed + } + + // -- RelationCatalog single-RPC perf path -- + + override def loadRelation(ident: Identifier): Table = { + val key = (ident.namespace().toSeq, ident.name()) + Option(views.get(key)) match { + case Some(info) => new MetadataOnlyTable(info, ident.toString) + case None => super.loadTable(ident) // delegate to InMemoryTableCatalog for tables + } + } +} + +object MetricViewRecordingCatalog { + // Captures every ViewInfo that flows through createView / replaceView so individual tests + // can assert on it. Cleared between tests via `reset()`. + val capturedViews: ConcurrentHashMap[Identifier, ViewInfo] = + new ConcurrentHashMap[Identifier, ViewInfo]() + + def reset(): Unit = capturedViews.clear() +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b05ee5abd033a..c4715b6a37efc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1091,7 +1091,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { "alternative" -> "DROP TABLE", "operation" -> "DROP VIEW", "foundType" -> "EXTERNAL", - "requiredType" -> "VIEW", + "requiredType" -> "VIEW or METRIC_VIEW", "objectName" -> "spark_catalog.dbx.tab1") ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index ff9be5ce759fe..8818983274ca0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1141,7 +1141,7 @@ class HiveDDLSuite "alternative" -> "DROP TABLE", "operation" -> "DROP VIEW", "foundType" -> "MANAGED", - "requiredType" -> "VIEW", + "requiredType" -> "VIEW or METRIC_VIEW", "objectName" -> s"$SESSION_CATALOG_NAME.default.tab1" ) ) From b5750535d5d506424133d84505b245f2650ed011 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Wed, 29 Apr 2026 15:50:15 +0000 Subject: [PATCH 02/17] [SPARK-XXXXX][SQL][FOLLOWUP] Address cloud-fan's metric-view review feedback Address all 4 substantive concerns and 8 inline comments from cloud-fan's review on PR #55487. V2 path refactor (concern 1, inline 87/111/142): - New CreateV2MetricViewExec extends V2ViewPreparation (mirrors CreateV2ViewExec): viewExists short-circuit on IF NOT EXISTS, OR REPLACE via createOrReplaceView, and cross-type collision decoding via ViewAlreadyExistsException -> tableExists -> EXPECT_VIEW_NOT_TABLE. - DataSourceV2Strategy rule routes CreateMetricViewCommand on a non-session catalog to the new exec; CreateMetricViewCommand keeps only the v1 session-catalog path in run(). - V2ViewPreparation gains optional viewDependencies / tableType hooks the metric-view subclass populates; plain views leave them None. - Picks up CharVarcharUtils.getRawSchema, SchemaUtils.checkColumn- NameDuplication, SchemaUtils.checkIndeterminateCollationInSchema, and PROP_OWNER stamping for free via the shared trait. Structural TableDependency / FunctionDependency (concern 3, inline 30/169): - Replace single-string `tableFullName` with `String[] nameParts`; arity is preserved per source, unambiguous against quoted identifiers containing literal `.`. Apply same shape to FunctionDependency. - Dependency.table / .function factories take varargs. - MetricViewHelper.collectTableDependencies returns Seq[Seq[String]]; each match arm emits structural parts (V1 sources via TableIdentifier.nameParts; V2 sources via catalog +: namespace :+ name). Sealed Dependency + defensive DependencyList (inline 30 / 40): - `sealed interface Dependency permits TableDependency, FunctionDependency` enforces the documented "is one of" structurally. - DependencyList canonical constructor and accessor defensively clone the array so consumers can't mutate stored ViewInfo dependency state. ViewInfo constructor cleanup (inline 68): - Drop the metric-view-specific PROP_TABLE_TYPE branch in the generic constructor in favor of `properties().putIfAbsent(...)`. Callers that want a more specific kind (e.g. METRIC_VIEW) call BaseBuilder.withTableType(...) before build() -- exercised by CreateV2MetricViewExec via the new V2ViewPreparation tableType hook. VIEW-gate audit (concern 2, inline 1088): - CatalogTable.toJsonLinkedHashMap (interface.scala:670) accepts METRIC_VIEW so DESCRIBE TABLE EXTENDED still emits "View Text" / "View Original Text" / "View Schema Mode" / "View Catalog and Namespace" / "SQL Path" rows for metric views. - HiveExternalCatalog: 4 sites (createTable empty-schema fallback x2, alterTable VIEW path, restoreTableMetadata read-back) accept METRIC_VIEW so a Hive-metastore-backed session-catalog metric view round-trips. - Repo-wide: SessionCatalog.isView, InMemoryCatalog.listViews, RelationResolution.createDataSourceV1Scan (streaming-on-view rejection), Analyzer.lookupTableOrView (ResolvedPersistentView wrapping), rules.saveDataIntoView, DataStreamWriter.writeToV1Table, DescribeRelationJsonCommand.describePartitionInfoJson, AnalyzeColumnCommand, AnalyzePartitionCommand, CommandUtils.analyzeTable, tables.CreateTableLike provider lookup, tables.AlterTableAddColumnsCommand, tables.describeDetailedPartitionInfo, tables.ShowCreateTableCommand (3 sites) -- all extended to accept METRIC_VIEW. Real error class for malformed YAML (concern 4, inline 70): - New INVALID_METRIC_VIEW_YAML error condition (sqlState 42K0L) + QueryCompilationErrors.invalidMetricViewYamlError. Replaces SparkException.internalError so a typo in the user's YAML body surfaces as a user-correctable AnalysisException. Tests (inline 39): - CREATE OR REPLACE VIEW WITH METRICS replaces the existing view. - CREATE VIEW IF NOT EXISTS WITH METRICS is a no-op when the view exists. - CREATE VIEW WITH METRICS over a v2 table at the ident throws EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE. - CREATE VIEW IF NOT EXISTS WITH METRICS is a no-op when a v2 table sits at the ident. - V1 session-catalog source dependency: nameParts arity 2-3. - Multi-level V2 namespace source dependency: nameParts arity N+2. - DESCRIBE TABLE EXTENDED on a v2 metric view -- read-back through loadRelation -> MetadataOnlyTable -> V1Table.toCatalogTable(ViewInfo). - Existing dep tests updated to assert nameParts instead of tableFullName. Misc (inline 178): - Scaladoc on MetricView.getProperties calls out that metric_view.from.sql / metric_view.where are truncated to Constants.MAXIMUM_PROPERTY_SIZE and are descriptive (not round-trippable) values; consumers should re-read the YAML body for full SQL. --- .../resources/error/error-conditions.json | 6 + .../sql/connector/catalog/Dependency.java | 25 +- .../sql/connector/catalog/DependencyList.java | 7 + .../connector/catalog/FunctionDependency.java | 20 +- .../connector/catalog/TableDependency.java | 28 ++- .../spark/sql/connector/catalog/ViewInfo.java | 11 +- .../sql/catalyst/analysis/Analyzer.scala | 3 +- .../analysis/RelationResolution.scala | 3 +- .../catalyst/catalog/InMemoryCatalog.scala | 5 +- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../sql/catalyst/catalog/interface.scala | 2 +- .../sql/errors/QueryCompilationErrors.scala | 7 + .../serde/MetricViewCanonical.scala | 9 + .../metricview/util/MetricViewPlanner.scala | 10 +- .../spark/sql/classic/DataStreamWriter.scala | 3 +- .../command/AnalyzeColumnCommand.scala | 3 +- .../command/AnalyzePartitionCommand.scala | 3 +- .../sql/execution/command/CommandUtils.scala | 3 +- .../command/DescribeRelationJsonCommand.scala | 3 +- .../command/metricViewCommands.scala | 103 ++------ .../spark/sql/execution/command/tables.scala | 19 +- .../sql/execution/datasources/rules.scala | 3 +- .../v2/CreateV2MetricViewExec.scala | 103 ++++++++ .../datasources/v2/CreateV2ViewExec.scala | 19 +- .../datasources/v2/DataSourceV2Strategy.scala | 32 ++- .../execution/MetricViewV2CatalogSuite.scala | 238 +++++++++++++++++- .../spark/sql/hive/HiveExternalCatalog.scala | 10 +- 27 files changed, 543 insertions(+), 138 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index b7de80510e223..cb38ffde4f46a 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4123,6 +4123,12 @@ }, "sqlState" : "KD002" }, + "INVALID_METRIC_VIEW_YAML" : { + "message" : [ + "Failed to parse metric view YAML: " + ], + "sqlState" : "42K0L" + }, "INVALID_NAME_IN_USE_COMMAND" : { "message" : [ "Invalid name '' in command. Reason: " diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java index b681b78f1033f..5a0ca939f98d8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java @@ -22,18 +22,31 @@ /** * Represents a dependency of a SQL object such as a view or metric view. *

- * A dependency is one of: {@link TableDependency} or {@link FunctionDependency}. + * A dependency is one of: {@link TableDependency} or {@link FunctionDependency}. The + * {@code sealed} declaration enforces this structurally. * * @since 4.2.0 */ @Evolving -public interface Dependency { +public sealed interface Dependency permits TableDependency, FunctionDependency { - static TableDependency table(String tableFullName) { - return new TableDependency(tableFullName); + /** + * Construct a {@link TableDependency} from the structural multi-part name of the dependent + * table. {@code nameParts} should contain at least one element; for catalog-managed tables + * the first element is typically the catalog name and subsequent elements are namespace + * components followed by the table name. + */ + static TableDependency table(String... nameParts) { + return new TableDependency(nameParts); } - static FunctionDependency function(String functionFullName) { - return new FunctionDependency(functionFullName); + /** + * Construct a {@link FunctionDependency} from the structural multi-part name of the + * dependent function. {@code nameParts} should contain at least one element; for + * catalog-managed functions the first element is typically the catalog name and subsequent + * elements are namespace components followed by the function name. + */ + static FunctionDependency function(String... nameParts) { + return new FunctionDependency(nameParts); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java index 6a1092bd94d73..1f64a67dc8cd9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java @@ -38,6 +38,13 @@ public record DependencyList(Dependency[] dependencies) { public DependencyList { Objects.requireNonNull(dependencies, "dependencies must not be null"); + dependencies = dependencies.clone(); + } + + /** Returns a defensive copy of the underlying dependencies array. */ + @Override + public Dependency[] dependencies() { + return dependencies.clone(); } public static DependencyList of(Dependency... dependencies) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java index dd76190788c83..7ad2522172b56 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java @@ -24,15 +24,25 @@ /** * A function dependency of a SQL object. *

- * The dependent function is identified by its fully-qualified three-part name - * in the form {@code catalog_name.schema_name.function_name}. + * The dependent function is identified by its structural multi-part name. See + * {@link TableDependency} for the parts-form contract. * - * @param functionFullName fully-qualified three-part function name + * @param nameParts structural multi-part identifier (defensive copy made; never empty) * @since 4.2.0 */ @Evolving -public record FunctionDependency(String functionFullName) implements Dependency { +public record FunctionDependency(String[] nameParts) implements Dependency { public FunctionDependency { - Objects.requireNonNull(functionFullName, "functionFullName must not be null"); + Objects.requireNonNull(nameParts, "nameParts must not be null"); + if (nameParts.length == 0) { + throw new IllegalArgumentException("nameParts must not be empty"); + } + nameParts = nameParts.clone(); + } + + /** Returns a defensive copy of the underlying parts array. */ + @Override + public String[] nameParts() { + return nameParts.clone(); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java index 5e44397139f46..32a7c462b4b98 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java @@ -24,15 +24,33 @@ /** * A table dependency of a SQL object. *

- * The dependent table is identified by its fully-qualified three-part name - * in the form {@code catalog_name.schema_name.table_name}. + * The dependent table is identified by its structural multi-part name. {@code nameParts} + * arity matches the catalog's namespace depth plus one for the table name -- for a catalog + * with single-level namespaces the parts are typically + * {@code [catalog, schema, table]}; for a catalog with multi-level namespaces (e.g. Iceberg + * with {@code db1.db2}) the parts are {@code [catalog, db1, db2, ..., table]}; for sources + * referenced through a session catalog without an explicit catalog component the parts can + * be {@code [db, table]} or just {@code [table]}. The structural form preserves arity and + * is unambiguous against quoted identifiers containing a literal {@code .}; consumers that + * need a flat string should join the parts themselves with a quoting scheme appropriate to + * their wire format. * - * @param tableFullName fully-qualified three-part table name + * @param nameParts structural multi-part identifier (defensive copy made; never empty) * @since 4.2.0 */ @Evolving -public record TableDependency(String tableFullName) implements Dependency { +public record TableDependency(String[] nameParts) implements Dependency { public TableDependency { - Objects.requireNonNull(tableFullName, "tableFullName must not be null"); + Objects.requireNonNull(nameParts, "nameParts must not be null"); + if (nameParts.length == 0) { + throw new IllegalArgumentException("nameParts must not be empty"); + } + nameParts = nameParts.clone(); + } + + /** Returns a defensive copy of the underlying parts array. */ + @Override + public String[] nameParts() { + return nameParts.clone(); } } 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 3710dd30b5b97..0f46e915a9be2 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 @@ -59,13 +59,10 @@ protected ViewInfo(Builder builder) { this.schemaMode = builder.schemaMode; this.queryColumnNames = builder.queryColumnNames; this.viewDependencies = builder.viewDependencies; - // Force PROP_TABLE_TYPE = VIEW by default so that `properties()` reflects the typed - // ViewInfo classification. Metric views intentionally set PROP_TABLE_TYPE = METRIC_VIEW - // before construction and must retain that more-specific view kind. - String tableType = properties().get(TableCatalog.PROP_TABLE_TYPE); - if (!TableSummary.METRIC_VIEW_TABLE_TYPE.equals(tableType)) { - properties().put(TableCatalog.PROP_TABLE_TYPE, TableSummary.VIEW_TABLE_TYPE); - } + // Default PROP_TABLE_TYPE = VIEW so `properties()` reflects the typed ViewInfo + // classification. Callers can refine to a more specific view kind (for example, + // METRIC_VIEW) by calling BaseBuilder.withTableType(...) on the builder before build(). + properties().putIfAbsent(TableCatalog.PROP_TABLE_TYPE, TableSummary.VIEW_TABLE_TYPE); } /** The SQL text of the view. */ 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 2d09759056c5a..17c21a9fbf8e9 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 @@ -1226,7 +1226,8 @@ class Analyzer( ) { CatalogV2Util.loadTable(catalog, ident).map { case v1Table: V1Table if CatalogV2Util.isSessionCatalog(catalog) && - v1Table.v1Table.tableType == CatalogTableType.VIEW => + (v1Table.v1Table.tableType == CatalogTableType.VIEW || + v1Table.v1Table.tableType == CatalogTableType.METRIC_VIEW) => val v1Ident = v1Table.catalogTable.identifier val v2Ident = Identifier.of(v1Ident.database.toArray, v1Ident.identifier) ResolvedPersistentView( 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 528e4ad0387a6..d69d9cc1492e7 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 @@ -398,7 +398,8 @@ class RelationResolution( timeTravelSpec: Option[TimeTravelSpec]): Option[LogicalPlan] = { def createDataSourceV1Scan(v1Table: CatalogTable): LogicalPlan = { if (isStreaming) { - if (v1Table.tableType == CatalogTableType.VIEW) { + if (v1Table.tableType == CatalogTableType.VIEW || + v1Table.tableType == CatalogTableType.METRIC_VIEW) { throw QueryCompilationErrors.permanentViewNotSupportedByStreamingReadingAPIError( ident.quoted ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index cd4a5645151b6..5536455c76c63 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -394,7 +394,10 @@ class InMemoryCatalog( override def listViews(db: String, pattern: String): Seq[String] = synchronized { requireDbExists(db) - val views = catalog(db).tables.filter(_._2.table.tableType == CatalogTableType.VIEW).keySet + val views = catalog(db).tables.filter { case (_, t) => + t.table.tableType == CatalogTableType.VIEW || + t.table.tableType == CatalogTableType.METRIC_VIEW + }.keySet StringUtils.filterPattern(views.toSeq.sorted, pattern) } 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 a3efade9b9a1c..41404b784580d 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 @@ -1256,7 +1256,8 @@ class SessionCatalog( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val ident = nameParts.asTableIdentifier try { - getTempViewOrPermanentTableMetadata(ident).tableType == CatalogTableType.VIEW + val t = getTempViewOrPermanentTableMetadata(ident).tableType + t == CatalogTableType.VIEW || t == CatalogTableType.METRIC_VIEW } catch { case _: NoSuchTableException => false case _: NoSuchNamespaceException => false 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 ed134589c2540..afe1406272373 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 @@ -667,7 +667,7 @@ case class CatalogTable( if (comment.isDefined) map += "Comment" -> JString(comment.get) if (collation.isDefined) map += "Collation" -> JString(collation.get) - if (tableType == CatalogTableType.VIEW) { + if (tableType == CatalogTableType.VIEW || tableType == CatalogTableType.METRIC_VIEW) { if (viewText.isDefined) { map += "View Text" -> JString(viewText.get) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 8c8fd4a4428be..31e975b5c333c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2826,6 +2826,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map.empty) } + def invalidMetricViewYamlError(message: String, cause: Throwable): Throwable = { + new AnalysisException( + errorClass = "INVALID_METRIC_VIEW_YAML", + messageParameters = Map("message" -> message), + cause = Some(cause)) + } + def noSuchStructFieldInGivenFieldsError( fieldName: String, fields: Array[StructField]): Throwable = { new AnalysisException( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala index 2a6f4d7c57c5c..1b4718ebd385e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/serde/MetricViewCanonical.scala @@ -174,6 +174,15 @@ private[sql] case class MetricView( * filter clauses. Mirrors the property keys used by the canonical metric view * representation on other Spark platforms so consumers of the catalog see a * consistent property layout. + * + * Note: `metric_view.from.sql` and `metric_view.where` values are truncated to + * [[Constants.MAXIMUM_PROPERTY_SIZE]] characters, so these are descriptive values + * for catalog browsers / lineage tooling -- not round-trippable representations + * of the source. Consumers that need the full SQL or filter expression for + * re-execution should read [[ViewInfo#queryText]] (the YAML body) and re-parse it + * rather than reconstruct the query from these properties; for any source whose + * SQL exceeds the size limit, this property would silently return a truncated + * string. */ def getProperties: Map[String, String] = { val base = Map(MetricView.PROP_FROM_TYPE -> from.sourceType.toString) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala index b2d39057750ea..2ac20ebeaa958 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/metricview/util/MetricViewPlanner.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.metricview.logical.MetricViewPlaceholder import org.apache.spark.sql.metricview.serde.{AssetSource, MetricView, MetricViewFactory, MetricViewValidationException, MetricViewYAMLParsingException, SQLSource} import org.apache.spark.sql.types.StructType @@ -63,12 +64,13 @@ object MetricViewPlanner { val metricView = try { MetricViewFactory.fromYAML(yaml) } catch { + // Both cases are user-correctable errors in the YAML body, not internal Spark bugs; + // surface them as `INVALID_METRIC_VIEW_YAML` AnalysisExceptions so the message is + // categorized as user input error rather than "please contact support". case e: MetricViewValidationException => - throw SparkException.internalError( - s"Invalid metric view YAML: ${e.getMessage}", e) + throw QueryCompilationErrors.invalidMetricViewYamlError(e.getMessage, e) case e: MetricViewYAMLParsingException => - throw SparkException.internalError( - s"Failed to parse metric view YAML: ${e.getMessage}", e) + throw QueryCompilationErrors.invalidMetricViewYamlError(e.getMessage, e) } val source = metricView.from match { case asset: AssetSource => UnresolvedRelation(sqlParser.parseMultipartIdentifier(asset.name)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala index 38483395ec8c5..fca3a31c3bd7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala @@ -190,7 +190,8 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) extends streaming.D val tableInstance = catalog.asTableCatalog.loadTable(identifier) def writeToV1Table(table: CatalogTable): StreamingQuery = { - if (table.tableType == CatalogTableType.VIEW) { + if (table.tableType == CatalogTableType.VIEW || + table.tableType == CatalogTableType.METRIC_VIEW) { throw QueryCompilationErrors.streamingIntoViewNotSupportedError(tableName) } require(table.provider.isDefined) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 157f0071a3dc6..15ebd77d07cdc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -104,7 +104,8 @@ case class AnalyzeColumnCommand( private def analyzeColumnInCatalog(sparkSession: SparkSession): Unit = { val sessionState = sparkSession.sessionState val tableMeta = sessionState.catalog.getTableMetadata(tableIdent) - if (tableMeta.tableType == CatalogTableType.VIEW) { + if (tableMeta.tableType == CatalogTableType.VIEW || + tableMeta.tableType == CatalogTableType.METRIC_VIEW) { // Analyzes a catalog view if the view is cached val plan = sparkSession.table(tableIdent.quotedString).logicalPlan if (!analyzeColumnInCachedData(plan, sparkSession)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala index 8f1e05c87c8f3..e6caf1a6b218c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala @@ -75,7 +75,8 @@ case class AnalyzePartitionCommand( val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) val tableMeta = sessionState.catalog.getTableMetadata(tableIdentWithDB) - if (tableMeta.tableType == CatalogTableType.VIEW) { + if (tableMeta.tableType == CatalogTableType.VIEW || + tableMeta.tableType == CatalogTableType.METRIC_VIEW) { throw QueryCompilationErrors.analyzeTableNotSupportedOnViewsError() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index 23055037ac4cf..559d6917d275c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -240,7 +240,8 @@ object CommandUtils extends Logging { val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) val tableMeta = sessionState.catalog.getTableMetadata(tableIdentWithDB) - if (tableMeta.tableType == CatalogTableType.VIEW) { + if (tableMeta.tableType == CatalogTableType.VIEW || + tableMeta.tableType == CatalogTableType.METRIC_VIEW) { // Analyzes a catalog view if the view is cached val table = sparkSession.table(tableIdent.quotedString) val cacheManager = sparkSession.sharedState.cacheManager 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 5f8528b679542..c035f0d3b16ad 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 @@ -309,7 +309,8 @@ case class DescribeRelationJsonCommand( catalog: SessionCatalog, metadata: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { - if (metadata.tableType == CatalogTableType.VIEW) { + if (metadata.tableType == CatalogTableType.VIEW || + metadata.tableType == CatalogTableType.METRIC_VIEW) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(metadata.identifier.identifier) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala index 6f16fb6041f77..2ab96749df91a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.command -import scala.jdk.CollectionConverters._ - import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{QueryPlanningTracker, TableIdentifier} @@ -26,11 +24,10 @@ import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, SchemaUnsuppo import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HiveTableRelation} import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, View} -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Dependency, DependencyList, TableCatalog, TableSummary, ViewCatalog, ViewInfo} +import org.apache.spark.sql.connector.catalog.CatalogV2Util import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.metricview.serde.MetricViewFactory import org.apache.spark.sql.metricview.util.MetricViewPlanner import org.apache.spark.sql.types.StructType @@ -47,10 +44,15 @@ case class CreateMetricViewCommand( override def run(sparkSession: SparkSession): Seq[Row] = { child match { - case v: ResolvedIdentifier if !CatalogV2Util.isSessionCatalog(v.catalog) => - createMetricViewInV2Catalog(sparkSession, v) - case v: ResolvedIdentifier => + case v: ResolvedIdentifier if CatalogV2Util.isSessionCatalog(v.catalog) => createMetricViewInSessionCatalog(sparkSession, v) + case _: ResolvedIdentifier => + // Non-session v2 catalogs are intercepted by `DataSourceV2Strategy` before the + // `ExecutedCommandExec(RunnableCommand)` fallback, so this branch is unreachable in + // practice. Keep the assertion for defensive clarity if a future strategy reorder + // accidentally exposes this code path. + throw SparkException.internalError( + "V2 metric-view CREATE should be handled by DataSourceV2Strategy, not run here") case _ => throw SparkException.internalError( s"Failed to resolve identifier for creating metric view") } @@ -84,65 +86,6 @@ case class CreateMetricViewCommand( Seq.empty } - private def createMetricViewInV2Catalog( - sparkSession: SparkSession, - resolved: ResolvedIdentifier): Seq[Row] = { - // Metric views are persisted through the same `ViewCatalog` interface as plain views; the - // only differences are `PROP_TABLE_TYPE = METRIC_VIEW` (so `V1Table.toCatalogTable` maps the - // round-tripped row back to `CatalogTableType.METRIC_VIEW`), the `metric_view.*` descriptor - // properties produced by `MetricView.getProperties`, and the typed `viewDependencies` field. - val viewCatalog = resolved.catalog match { - case vc: ViewCatalog => vc - case other => - throw QueryCompilationErrors.missingCatalogViewsAbilityError(other) - } - val ident = resolved.identifier - val name = ident.asTableIdentifier - - val analyzed = MetricViewHelper.analyzeMetricViewText(sparkSession, name, originalText) - validateUserColumns(name, analyzed) - - // `retainMetadata = true` preserves the per-column `metric_view.type` / `metric_view.expr` - // metadata that `ResolveMetricView.buildMetricViewOutput` attaches, even when the user - // supplies column names with comments (same contract as the V1 session-catalog path, which - // goes through `ViewHelper.prepareTable` with `isMetricView = true`). - val aliasedSchema = ViewHelper - .aliasPlan(sparkSession, analyzed, userSpecifiedColumns, retainMetadata = true) - .schema - - // Describe this metric view's source and filter as user-visible properties so catalogs and - // tooling can inspect them without re-parsing the YAML body. - val metricView = MetricViewFactory.fromYAML(originalText) - val viewProperties = new java.util.HashMap[String, String]() - properties.foreach { case (k, v) => viewProperties.put(k, v) } - metricView.getProperties.foreach { case (k, v) => viewProperties.put(k, v) } - viewProperties.put(TableCatalog.PROP_TABLE_TYPE, TableSummary.METRIC_VIEW_TABLE_TYPE) - - val sourceTableNames = MetricViewHelper.collectTableDependencies(analyzed) - val deps = if (sourceTableNames.nonEmpty) { - DependencyList.of(sourceTableNames.map(Dependency.table): _*) - } else { - null - } - - val manager = sparkSession.sessionState.catalogManager - val builder = new ViewInfo.Builder() - .withSchema(aliasedSchema) - .withProperties(viewProperties) - .withQueryText(originalText) - .withCurrentCatalog(manager.currentCatalog.name) - .withCurrentNamespace(manager.currentNamespace) - .withSqlConfigs( - ViewHelper.sqlConfigsToProps(sparkSession.sessionState.conf, "").asJava) - .withSchemaMode(SchemaUnsupported.toString) - .withQueryColumnNames(analyzed.output.map(_.name).toArray) - .withViewDependencies(deps) - comment.foreach(builder.withComment) - - viewCatalog.createView(ident, builder.build()) - Seq.empty - } - override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = { copy(child = newChild) } @@ -153,24 +96,28 @@ case class AlterMetricViewCommand(child: LogicalPlan, originalText: String) object MetricViewHelper { /** - * Walks the analyzed plan to collect direct table/view dependencies. - * Stops recursion at relation leaf nodes and persistent View nodes so that only - * direct (not transitive) dependencies are recorded. + * Walks the analyzed plan to collect direct table/view dependencies. Each dependency is + * returned as a structural multi-part name (`Seq[String]`); arity is preserved per source + * so consumers can reason about catalog / namespace / table boundaries without parsing a + * dot-flattened string. + * + * Stops recursion at relation leaf nodes and persistent `View` nodes so only direct + * (not transitive) dependencies are recorded. */ - private[execution] def collectTableDependencies(plan: LogicalPlan): Seq[String] = { - val tables = scala.collection.mutable.ArrayBuffer.empty[String] + private[execution] def collectTableDependencies(plan: LogicalPlan): Seq[Seq[String]] = { + val tables = scala.collection.mutable.ArrayBuffer.empty[Seq[String]] def traverse(p: LogicalPlan): Unit = p match { case v: View if !v.isTempView => - tables += v.desc.identifier.unquotedString + tables += v.desc.identifier.nameParts case r: DataSourceV2Relation if r.catalog.isDefined && r.identifier.isDefined => - val cat = r.catalog.get.name() - val ns = r.identifier.get.namespace().mkString(".") - val name = r.identifier.get.name() - tables += s"$cat.$ns.$name" + val ident = r.identifier.get + // V2 catalogs may have multi-level namespaces; preserve the full arity rather than + // dot-joining the namespace into a single component. + tables += (r.catalog.get.name() +: ident.namespace().toIndexedSeq) :+ ident.name() case r: HiveTableRelation => - tables += r.tableMeta.identifier.unquotedString + tables += r.tableMeta.identifier.nameParts case r: LogicalRelation if r.catalogTable.isDefined => - tables += r.catalogTable.get.identifier.unquotedString + tables += r.catalogTable.get.identifier.nameParts case other => other.children.foreach(traverse) other.expressions.foreach(_.foreach { 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 fb2dc0a684943..da8662178e29f 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 @@ -103,7 +103,8 @@ case class CreateTableLikeCommand( provider } else if (fileFormat.inputFormat.isDefined) { Some(DDLUtils.HIVE_PROVIDER) - } else if (sourceTableDesc.tableType == CatalogTableType.VIEW) { + } else if (sourceTableDesc.tableType == CatalogTableType.VIEW || + sourceTableDesc.tableType == CatalogTableType.METRIC_VIEW) { Some(sparkSession.sessionState.conf.defaultDataSourceName) } else { sourceTableDesc.provider @@ -267,7 +268,8 @@ case class AlterTableAddColumnsCommand( table: TableIdentifier): CatalogTable = { val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) - if (catalogTable.tableType == CatalogTableType.VIEW) { + if (catalogTable.tableType == CatalogTableType.VIEW || + catalogTable.tableType == CatalogTableType.METRIC_VIEW) { throw QueryCompilationErrors.alterAddColNotSupportViewError(table) } @@ -730,7 +732,8 @@ case class DescribeTableCommand( catalog: SessionCatalog, metadata: CatalogTable, result: ArrayBuffer[Row]): Unit = { - if (metadata.tableType == CatalogTableType.VIEW) { + if (metadata.tableType == CatalogTableType.VIEW || + metadata.tableType == CatalogTableType.METRIC_VIEW) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(table.identifier) } DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION") @@ -1226,7 +1229,7 @@ case class ShowCreateTableCommand( tableMetadata) } - if (tableMetadata.tableType == VIEW) { + if (tableMetadata.tableType == VIEW || tableMetadata.tableType == METRIC_VIEW) { tableMetadata } else { convertTableMetadata(tableMetadata) @@ -1235,7 +1238,11 @@ case class ShowCreateTableCommand( val builder = new StringBuilder - val stmt = if (tableMetadata.tableType == VIEW) { + // SHOW CREATE TABLE on a metric view falls through to the VIEW branch, which emits + // `CREATE VIEW ...` without the `WITH METRICS` qualifier. The output is not yet + // round-trippable for metric views; tracked as follow-up. + val stmt = if (tableMetadata.tableType == VIEW || + tableMetadata.tableType == METRIC_VIEW) { builder ++= s"CREATE VIEW ${table.quoted} " showCreateView(metadata, builder) @@ -1386,7 +1393,7 @@ case class ShowCreateTableAsSerdeCommand( builder ++= s"CREATE$tableTypeString ${table.quoted} " - if (metadata.tableType == VIEW) { + if (metadata.tableType == VIEW || metadata.tableType == METRIC_VIEW) { showCreateView(metadata, builder) } else { showHiveTableHeader(metadata, builder) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index d1f61599e7ac8..44b13fdcb3de9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -169,7 +169,8 @@ case class PreprocessTableCreation(catalog: SessionCatalog) extends Rule[Logical val tableName = tableIdentWithDB.unquotedString val existingTable = catalog.getTableMetadata(tableIdentWithDB) - if (existingTable.tableType == CatalogTableType.VIEW) { + if (existingTable.tableType == CatalogTableType.VIEW || + existingTable.tableType == CatalogTableType.METRIC_VIEW) { throw QueryCompilationErrors.saveDataIntoViewNotAllowedError() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala new file mode 100644 index 0000000000000..50d6ba1e6f4e6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala @@ -0,0 +1,103 @@ +/* + * 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 org.apache.spark.sql.catalyst.{CurrentUserContext, InternalRow} +import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, SchemaUnsupported, ViewAlreadyExistsException, ViewSchemaMode} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{DependencyList, Identifier, TableCatalog, TableSummary, ViewCatalog} +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.command.CommandUtils + +/** + * Physical plan node for `CREATE VIEW ... WITH METRICS` on a v2 [[ViewCatalog]]. Mirrors + * [[CreateV2ViewExec]]'s flag handling and cross-type collision decoding via the shared + * [[V2ViewPreparation]] trait, and adds metric-view-specific bits (typed + * [[DependencyList]] view-dependencies and `PROP_TABLE_TYPE = METRIC_VIEW`) through the + * trait's optional hooks. + * + * Routed by [[DataSourceV2Strategy]] from + * [[org.apache.spark.sql.execution.command.CreateMetricViewCommand]] when the resolved + * catalog is a non-session v2 catalog. + */ +case class CreateV2MetricViewExec( + catalog: ViewCatalog, + identifier: Identifier, + userSpecifiedColumns: Seq[(String, Option[String])], + comment: Option[String], + userProperties: Map[String, String], + originalText: String, + query: LogicalPlan, + allowExisting: Boolean, + replace: Boolean, + deps: Option[DependencyList]) extends V2ViewPreparation { + + // Metric views don't carry a default-collation override. + override def collation: Option[String] = None + + // CREATE stamps the current user, matching the v1 metric-view path (which goes through + // ViewHelper.prepareTable -> CatalogTable.owner default) and CreateV2ViewExec. + override def owner: Option[String] = Some(CurrentUserContext.getCurrentUser) + + // Metric views always have schema-mode UNSUPPORTED (mirroring the v1 path which passes + // SchemaUnsupported into ViewHelper.prepareTable). + override def viewSchemaMode: ViewSchemaMode = SchemaUnsupported + + override protected def viewDependencies: Option[DependencyList] = deps + + override protected def tableType: Option[String] = + Some(TableSummary.METRIC_VIEW_TABLE_TYPE) + + override protected def run(): Seq[InternalRow] = { + // CREATE VIEW IF NOT EXISTS short-circuit, identical to CreateV2ViewExec: skip the + // `buildViewInfo` work when a view already sits at the ident. The mixed-catalog + // "table at ident" no-op is handled in the catch block below. + if (allowExisting && catalog.viewExists(identifier)) return Seq.empty + + val info = buildViewInfo() + try { + if (replace) { + CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) + catalog.createOrReplaceView(identifier, info) + } else { + catalog.createView(identifier, info) + } + } catch { + case _: ViewAlreadyExistsException => + // Catalog refused: decode whether a table sits at the ident (cross-type collision) + // or a view (race for plain CREATE / OR REPLACE), and emit the precise error -- or + // no-op for IF NOT EXISTS. Same shape as CreateV2ViewExec. + val isTable = catalog match { + case tc: TableCatalog => tc.tableExists(identifier) + case _ => false + } + if (isTable) { + if (!allowExisting) { + throw QueryCompilationErrors.unsupportedCreateOrReplaceViewOnTableError( + fullNameParts, replace) + } + // CREATE VIEW IF NOT EXISTS over a table is a no-op (v1 parity). + } else if (!allowExisting) { + throw viewAlreadyExists() + } + // else: a view appeared between our viewExists probe and createView; IF NOT EXISTS + // semantics make this a no-op. + } + Seq.empty + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index 6cfa95a2eaf43..f046d93219f30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, SchemaEvoluti import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CharVarcharUtils -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, ViewCatalog, ViewInfo} +import org.apache.spark.sql.connector.catalog.{DependencyList, 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, ViewHelper} @@ -57,6 +57,21 @@ private[v2] trait V2ViewPreparation extends LeafV2CommandExec { protected lazy val fullNameParts: Seq[String] = (catalog.name() +: identifier.asMultipartIdentifier).toSeq + /** + * Optional structured dependency list to stamp on the built `ViewInfo`. Plain views leave + * this `None`; metric views populate it with the source-table lineage produced by + * `MetricViewHelper.collectTableDependencies`. + */ + protected def viewDependencies: Option[DependencyList] = None + + /** + * Optional view sub-kind to stamp on `PROP_TABLE_TYPE`. Plain views leave this `None` so + * the `ViewInfo` constructor's `putIfAbsent` defaults it to `VIEW`; metric views set it to + * `TableSummary.METRIC_VIEW_TABLE_TYPE` so consumers (e.g. `V1Table.toCatalogTable`) can + * round-trip the discriminator. + */ + protected def tableType: Option[String] = None + override def output: Seq[Attribute] = Seq.empty protected def buildViewInfo(): ViewInfo = { @@ -106,6 +121,8 @@ private[v2] trait V2ViewPreparation extends LeafV2CommandExec { owner.foreach(builder.withOwner) comment.foreach(builder.withComment) collation.foreach(builder.withCollation) + viewDependencies.foreach(builder.withViewDependencies) + tableType.foreach(builder.withTableType) builder.build() } 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 eacb97a069f65..58065c1793ec4 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 @@ -34,7 +34,7 @@ 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.{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, V1ViewInfo, ViewCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Dependency, DependencyList, 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} @@ -44,11 +44,12 @@ import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBat import org.apache.spark.sql.connector.write.V1Write import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.{FilterExec, InSubqueryExec, LeafExecNode, LocalTableScanExec, ProjectExec, RowDataSourceScanExec, ScalarSubquery => ExecScalarSubquery, SparkPlan, SparkStrategy => Strategy} -import org.apache.spark.sql.execution.command.CommandUtils +import org.apache.spark.sql.execution.command.{CommandUtils, CreateMetricViewCommand, MetricViewHelper} import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRelationWithTable, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH +import org.apache.spark.sql.metricview.serde.MetricViewFactory import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ArrayImplicits._ @@ -323,6 +324,33 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat CreateV2ViewExec(catalog.asInstanceOf[ViewCatalog], ident, userSpecifiedColumns, comment, collation, properties, sqlText, child, allowExisting, replace, viewSchemaMode) :: Nil + // CREATE VIEW ... WITH METRICS on a non-session v2 catalog. Routes the metric-view path + // through `CreateV2MetricViewExec`, which extends `V2ViewPreparation` to share the + // `IF NOT EXISTS` short-circuit, `OR REPLACE`, and cross-type-collision decoding with + // `CreateV2ViewExec`. Session-catalog dispatch stays in `CreateMetricViewCommand.run`. + case CreateMetricViewCommand( + ResolvedIdentifier(catalog, ident), userSpecifiedColumns, comment, properties, + originalText, allowExisting, replace) if !CatalogV2Util.isSessionCatalog(catalog) => + val viewCatalog = catalog match { + case vc: ViewCatalog => vc + case _ => throw QueryCompilationErrors.missingCatalogViewsAbilityError(catalog) + } + // Parse + analyze the YAML body here (during planning). This mirrors the v1 path's + // late analysis in `CreateMetricViewCommand.run` -- the metric-view source plan is not + // a SQL string, so it can't ride along as a regular `query` `LogicalPlan` field on the + // logical command the way `CreateView` does. + val analyzed = MetricViewHelper.analyzeMetricViewText( + session, ident.asTableIdentifier, originalText) + val metricView = MetricViewFactory.fromYAML(originalText) + val mergedProps = properties ++ metricView.getProperties + val depParts = MetricViewHelper.collectTableDependencies(analyzed) + val deps = if (depParts.nonEmpty) { + Some(DependencyList.of( + depParts.map(parts => Dependency.table(parts: _*)): _*)) + } else None + CreateV2MetricViewExec(viewCatalog, ident, userSpecifiedColumns, comment, mergedProps, + originalText, analyzed, allowExisting, replace, deps) :: Nil + case AlterViewAs(rpv @ ResolvedPersistentView(catalog, ident, _), originalText, query, _, _) => AlterV2ViewExec(catalog.asInstanceOf[ViewCatalog], ident, rpv.info, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 18acc82957257..4e0e766289980 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -138,7 +138,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { assert(deps != null) assert(deps.dependencies().length === 1) val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.tableFullName() === fullSourceTableName) + assert(tableDep.nameParts().toSeq === + Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -216,7 +217,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val deps = info.viewDependencies() assert(deps != null && deps.dependencies().length === 1) val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.tableFullName() === fullSourceTableName) + assert(tableDep.nameParts().toSeq === + Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -307,10 +309,12 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val deps = capturedViewInfo().viewDependencies() assert(deps != null) - val depNames = - deps.dependencies().map(_.asInstanceOf[TableDependency].tableFullName()).toSet - assert(depNames === Set(fullSourceTableName, secondSource), - s"Expected dependencies on both source tables, got $depNames") + val depParts = + deps.dependencies().map(_.asInstanceOf[TableDependency].nameParts().toSeq).toSet + assert(depParts === Set( + Seq(testCatalogName, testNamespace, sourceTableName), + Seq(testCatalogName, testNamespace, "customers")), + s"Expected dependencies on both source tables, got $depParts") } finally { sql(s"DROP TABLE IF EXISTS $secondSource") } @@ -334,7 +338,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { s"Expected 1 deduplicated dependency, got " + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.tableFullName() === fullSourceTableName) + assert(tableDep.nameParts().toSeq === + Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -358,12 +363,227 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { s"Expected 1 deduplicated dependency for self-join, got " + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.tableFullName() === fullSourceTableName) + assert(tableDep.nameParts().toSeq === + Seq(testCatalogName, testNamespace, sourceTableName)) + } + } + + test("CREATE OR REPLACE VIEW ... WITH METRICS replaces an existing v2 metric view") { + withTestCatalogTables { + val first = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = Some("count > 0"), + select = metricViewColumns) + createMetricView(fullMetricViewName, first) + val firstYaml = capturedViewInfo().queryText() + + // Replace with a new body (different WHERE clause). + val replacement = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = Some("count > 100"), + select = metricViewColumns) + val replacementYaml = MetricViewFactory.toYAML(replacement) + sql( + s"""CREATE OR REPLACE VIEW $fullMetricViewName + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |$replacementYaml + |$$$$""".stripMargin) + + val finalInfo = capturedViewInfo() + assert(finalInfo.queryText() === replacementYaml, + "OR REPLACE should swap the captured ViewInfo's queryText.") + assert(finalInfo.queryText() !== firstYaml, + "OR REPLACE should not leave the original captured queryText in place.") + } + } + + test("CREATE VIEW IF NOT EXISTS ... WITH METRICS is a no-op when the view exists") { + withTestCatalogTables { + val original = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, original) + val originalYaml = capturedViewInfo().queryText() + + // Now CREATE VIEW IF NOT EXISTS with a different YAML body. The catalog should not see + // the second create at all (V2ViewPreparation's `viewExists` short-circuit fires before + // `buildViewInfo`), so the captured ViewInfo retains the original body. + val replacement = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = Some("count > 999"), + select = metricViewColumns) + val replacementYaml = MetricViewFactory.toYAML(replacement) + sql( + s"""CREATE VIEW IF NOT EXISTS $fullMetricViewName + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |$replacementYaml + |$$$$""".stripMargin) + + assert(capturedViewInfo().queryText() === originalYaml, + "IF NOT EXISTS over an existing metric view should be a no-op.") + } + } + + test("CREATE VIEW ... WITH METRICS over a v2 table at the ident throws " + + "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE") { + withTestCatalogTables { + // Pre-create a regular v2 table at the same ident the metric view will target. The + // catalog's `createView` call below should raise `ViewAlreadyExistsException`, which + // `CreateV2MetricViewExec` then decodes (via `tableExists`) into the precise cross-type + // collision error that `CreateV2ViewExec` emits. + sql(s"CREATE TABLE $fullMetricViewName (x INT) USING foo") + + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + val yaml = MetricViewFactory.toYAML(mv) + val ex = intercept[AnalysisException] { + sql( + s"""CREATE VIEW $fullMetricViewName + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |$yaml + |$$$$""".stripMargin) + } + // CreateV2ViewExec / CreateV2MetricViewExec route this through + // `unsupportedCreateOrReplaceViewOnTableError` which maps to + // `EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE`. + assert(ex.getCondition === "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE", + s"Expected EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE, got ${ex.getCondition}: ${ex.getMessage}") + } + } + + test("CREATE VIEW IF NOT EXISTS ... WITH METRICS is a no-op when a v2 table sits at the " + + "ident") { + withTestCatalogTables { + sql(s"CREATE TABLE $fullMetricViewName (x INT) USING foo") + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + val yaml = MetricViewFactory.toYAML(mv) + // IF NOT EXISTS over a table is a no-op (v1 parity), not an error. + sql( + s"""CREATE VIEW IF NOT EXISTS $fullMetricViewName + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |$yaml + |$$$$""".stripMargin) + val ident = Identifier.of(Array(testNamespace), metricViewName) + assert(!MetricViewRecordingCatalog.capturedViews.containsKey(ident), + "IF NOT EXISTS over a v2 table should not register a view in the catalog.") + } + } + + test("dependency extraction: V1 session-catalog source emits 3-part nameParts") { + val v1Source = "metric_view_v2_v1source" + spark.range(0, 5).toDF("v") + .write.mode("overwrite").saveAsTable(v1Source) + try { + withTestCatalogTables { + val mv = MetricView( + "0.1", + // SQL source resolves through the current (session) catalog; the resolved + // `LogicalRelation` carries a session-catalog `CatalogTable`. + SQLSource(s"SELECT v AS region, v AS count FROM $v1Source"), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null && deps.dependencies().length === 1) + val parts = deps.dependencies()(0).asInstanceOf[TableDependency].nameParts().toSeq + // For a session-catalog source, `TableIdentifier.nameParts` includes catalog + db + + // table when the catalog is set; here we expect at least 2 parts (`db.table`) and + // up to 3 (`spark_catalog.db.table`) -- both are valid producer outputs depending + // on whether the analyzer captured the session-catalog component. + assert(parts.last === v1Source, s"Last part should be the table name, got $parts") + assert(parts.length >= 2 && parts.length <= 3, + s"V1 nameParts arity should be 2 or 3, got ${parts.length}: $parts") + } + } finally { + sql(s"DROP TABLE IF EXISTS $v1Source") + } + } + + test("dependency extraction: multi-level V2 namespace source emits N+2 nameParts") { + val multiNamespace = Array("ns_a", "ns_b") + val multiTable = "events_deep" + val multiFull = s"$testCatalogName.${multiNamespace.mkString(".")}.$multiTable" + withTestCatalogTables { + // The InMemoryTableCatalog (RelationCatalog mixin) supports multi-level namespaces. + sql(s"CREATE NAMESPACE IF NOT EXISTS $testCatalogName.${multiNamespace.head}") + sql(s"CREATE NAMESPACE IF NOT EXISTS " + + s"$testCatalogName.${multiNamespace.mkString(".")}") + sql(s"CREATE TABLE $multiFull (region STRING, count INT) USING foo") + try { + val mv = MetricView( + "0.1", + SQLSource(s"SELECT region, count FROM $multiFull"), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null && deps.dependencies().length === 1) + val parts = deps.dependencies()(0).asInstanceOf[TableDependency].nameParts().toSeq + assert(parts === Seq(testCatalogName, multiNamespace(0), multiNamespace(1), multiTable), + s"Multi-level nameParts should preserve every namespace component, got $parts") + } finally { + sql(s"DROP TABLE IF EXISTS $multiFull") + sql(s"DROP NAMESPACE IF EXISTS " + + s"$testCatalogName.${multiNamespace.mkString(".")} CASCADE") + sql(s"DROP NAMESPACE IF EXISTS $testCatalogName.${multiNamespace.head} CASCADE") + } + } + } + + test("DESCRIBE TABLE EXTENDED on a v2 metric view round-trips through loadRelation") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + val yaml = createMetricView(fullMetricViewName, mv) + + // DESCRIBE TABLE EXTENDED resolves the ident through `Analyzer.lookupTableOrView`, + // which calls `RelationCatalog.loadRelation` once and gets back a + // `MetadataOnlyTable(ViewInfo)`. `V1Table.toCatalogTable(ViewInfo)` reconstructs the + // V1 representation; the resulting `CatalogTable.toJsonLinkedHashMap` (interface.scala) + // then emits view-context rows because `tableType == METRIC_VIEW` was added to the + // VIEW gate. Without that gate fix, "View Text" / "View Original Text" disappear. + val rows = sql(s"DESCRIBE TABLE EXTENDED $fullMetricViewName").collect() + val rowMap = rows.map(r => r.getString(0) -> r.getString(1)).toMap + + assert(rowMap.contains("View Text"), + s"Expected 'View Text' row in DESCRIBE EXTENDED output, got keys: ${rowMap.keys}") + assert(rowMap("View Text") === yaml, + s"View Text should round-trip the YAML body, got: ${rowMap("View Text")}") + assert(rowMap.get("Type").exists(_.contains("METRIC_VIEW")), + s"Type row should reflect METRIC_VIEW, got: ${rowMap.get("Type")}") } } test("CREATE VIEW ... WITH METRICS on a non-ViewCatalog catalog fails with " + - "MISSING_CATALOG_ABILITY.VIEWS") { val ex = intercept[AnalysisException] { sql( s"""CREATE VIEW ${MetricViewV2CatalogSuite.noViewCatalogName}.default.mv diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 8ec4f97c43e85..a72fbce74be1c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -274,7 +274,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // Spark-created views do not have to be Hive compatible. If the data type is not // Hive compatible, we can set schema to empty so that Spark can still read this // view as the schema is also encoded in the table properties. - case schema if tableDefinition.tableType == CatalogTableType.VIEW && + case schema if (tableDefinition.tableType == CatalogTableType.VIEW || + tableDefinition.tableType == CatalogTableType.METRIC_VIEW) && schema.exists(f => !isHiveCompatibleDataType(f.dataType)) => EMPTY_DATA_SCHEMA case other => other @@ -294,7 +295,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat try { client.createTable(tableWithDataSourceProps, ignoreIfExists) } catch { - case NonFatal(e) if tableDefinition.tableType == CatalogTableType.VIEW && + case NonFatal(e) if (tableDefinition.tableType == CatalogTableType.VIEW || + tableDefinition.tableType == CatalogTableType.METRIC_VIEW) && hiveCompatibleSchema != EMPTY_DATA_SCHEMA => // If for some reason we fail to store the schema we store it as empty there // since we already store the real schema in the table properties. This try-catch @@ -595,7 +597,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat requireTableExists(db, tableDefinition.identifier.table) verifyTableProperties(tableDefinition) - if (tableDefinition.tableType == VIEW) { + if (tableDefinition.tableType == VIEW || tableDefinition.tableType == METRIC_VIEW) { val newTableProps = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition).toMap val schemaWithNoCollation = removeCollation(tableDefinition.schema) val hiveCompatibleSchema = @@ -835,7 +837,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat var table = inputTable table.properties.get(DATASOURCE_PROVIDER) match { - case None if table.tableType == VIEW => + case None if table.tableType == VIEW || table.tableType == METRIC_VIEW => // If this is a view created by Spark 2.2 or higher versions, we should restore its schema // from table properties. getSchemaFromTableProperties(table.properties).foreach { schemaFromTableProps => From 0509d5bac33af8e12b78687f576e86c087f4b639 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Wed, 29 Apr 2026 20:01:26 +0000 Subject: [PATCH 03/17] [SPARK-XXXXX][SQL][FOLLOWUP] Address self-review + fix CI on PR #55487 CI fix: - Restore the missing `"MISSING_CATALOG_ABILITY.VIEWS") {` line in `MetricViewV2CatalogSuite.scala` whose absence broke the test file's parse (compileIncremental "four errors found" -- ')' expected, unmatched '}', unclosed multi-line string, '}' expected at EOF). Self-review feedback: - Shorten `viewDependencies` and `tableType` Scaladocs on `V2ViewPreparation` to one sentence each; drop the plain-view / metric-view distinctions. - Consolidate the duplicated `run()` body from `CreateV2ViewExec` and `CreateV2MetricViewExec` into a new `V2CreateViewPreparation` trait (extends `V2ViewPreparation`) that owns the shared CREATE-side flow: `viewExists` short-circuit, `OR REPLACE` via `createOrReplaceView`, and `ViewAlreadyExistsException` decoding for cross-type collisions. Both subclasses now extend `V2CreateViewPreparation` and inherit `run()` unchanged. The intermediate trait keeps `V2ViewPreparation` (also used by `AlterV2ViewExec`) free of CREATE-only fields. - Drop the defensive `case _: ResolvedIdentifier =>` branch in `CreateMetricViewCommand.run` -- the catch-all internal-error case is sufficient now that `DataSourceV2Strategy` reliably intercepts the non-session path. - `SHOW CREATE TABLE` on a metric view now throws the dedicated `UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW` (sqlState 0A000) instead of falling through to the VIEW branch. Reverts the three METRIC_VIEW additions in `tables.scala`'s VIEW gates from the previous round. Test reorg + new coverage in `MetricViewV2CatalogSuite`: - Reorganized into 5 sections: (1) Create-related, (2) Dependency extraction, (3) SELECT cases, (4) DESCRIBE cases, (5) DROP / SHOW. - Trimmed the `OR REPLACE` test's `!== firstYaml` assertion; replaced with positive assertions on the captured ViewInfo's queryText, metric_view.where, and viewDependencies (parts shape). - New `SELECT from a v2 metric view returns aggregated rows` -- exercises the `loadRelation` -> `MetadataOnlyTable(ViewInfo)` -> `V1Table.toCatalogTable(ViewInfo)` -> `ResolveMetricView` round-trip and asserts on the aggregated output rows. - New `DESCRIBE TABLE on a v2 metric view returns the aliased columns` (non-EXTENDED variant alongside the existing EXTENDED test). - New `SHOW TABLES does not list v2 metric views` (RelationCatalog contract: tables only). - New `SHOW VIEWS lists v2 metric views`. --- .../resources/error/error-conditions.json | 5 + .../sql/errors/QueryCompilationErrors.scala | 6 + .../command/metricViewCommands.scala | 7 - .../spark/sql/execution/command/tables.scala | 18 +- .../v2/CreateV2MetricViewExec.scala | 58 +-- .../datasources/v2/CreateV2ViewExec.scala | 65 ++-- .../execution/MetricViewV2CatalogSuite.scala | 348 +++++++++++------- 7 files changed, 290 insertions(+), 217 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index cb38ffde4f46a..1aaebc60e1a17 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -8267,6 +8267,11 @@ "The table is a Spark data source table. Please use SHOW CREATE TABLE without AS SERDE instead." ] }, + "ON_METRIC_VIEW" : { + "message" : [ + "The command is not supported on a metric view ." + ] + }, "ON_TEMPORARY_VIEW" : { "message" : [ "The command is not supported on a temporary view ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 31e975b5c333c..c2f7d4ebcf46a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -3330,6 +3330,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map("tableName" -> toSQLId(table))) } + def showCreateTableNotSupportedOnMetricViewError(table: String): Throwable = { + new AnalysisException( + errorClass = "UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW", + messageParameters = Map("tableName" -> toSQLId(table))) + } + def showCreateTableNotSupportTransactionalHiveTableError(table: CatalogTable): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_SHOW_CREATE_TABLE.ON_TRANSACTIONAL_HIVE_TABLE", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala index 2ab96749df91a..ed3a3a2a223c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala @@ -46,13 +46,6 @@ case class CreateMetricViewCommand( child match { case v: ResolvedIdentifier if CatalogV2Util.isSessionCatalog(v.catalog) => createMetricViewInSessionCatalog(sparkSession, v) - case _: ResolvedIdentifier => - // Non-session v2 catalogs are intercepted by `DataSourceV2Strategy` before the - // `ExecutedCommandExec(RunnableCommand)` fallback, so this branch is unreachable in - // practice. Keep the assertion for defensive clarity if a future strategy reorder - // accidentally exposes this code path. - throw SparkException.internalError( - "V2 metric-view CREATE should be handled by DataSourceV2Strategy, not run here") case _ => throw SparkException.internalError( s"Failed to resolve identifier for creating metric view") } 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 da8662178e29f..59068463497f9 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 @@ -1213,6 +1213,14 @@ case class ShowCreateTableCommand( } else { val tableMetadata = catalog.getTableRawMetadata(table) + // SHOW CREATE TABLE / VIEW does not have a WITH METRICS round-trippable form yet, + // so explicitly reject metric views rather than emit a misleading `CREATE VIEW` + // statement that loses the METRIC_VIEW kind. Tracked as follow-up. + if (tableMetadata.tableType == METRIC_VIEW) { + throw QueryCompilationErrors.showCreateTableNotSupportedOnMetricViewError( + table.identifier) + } + // TODO: [SPARK-28692] unify this after we unify the // CREATE TABLE syntax for hive serde and data source table. val metadata = if (DDLUtils.isDatasourceTable(tableMetadata)) { @@ -1229,7 +1237,7 @@ case class ShowCreateTableCommand( tableMetadata) } - if (tableMetadata.tableType == VIEW || tableMetadata.tableType == METRIC_VIEW) { + if (tableMetadata.tableType == VIEW) { tableMetadata } else { convertTableMetadata(tableMetadata) @@ -1238,11 +1246,7 @@ case class ShowCreateTableCommand( val builder = new StringBuilder - // SHOW CREATE TABLE on a metric view falls through to the VIEW branch, which emits - // `CREATE VIEW ...` without the `WITH METRICS` qualifier. The output is not yet - // round-trippable for metric views; tracked as follow-up. - val stmt = if (tableMetadata.tableType == VIEW || - tableMetadata.tableType == METRIC_VIEW) { + val stmt = if (tableMetadata.tableType == VIEW) { builder ++= s"CREATE VIEW ${table.quoted} " showCreateView(metadata, builder) @@ -1393,7 +1397,7 @@ case class ShowCreateTableAsSerdeCommand( builder ++= s"CREATE$tableTypeString ${table.quoted} " - if (metadata.tableType == VIEW || metadata.tableType == METRIC_VIEW) { + if (metadata.tableType == VIEW) { showCreateView(metadata, builder) } else { showHiveTableHeader(metadata, builder) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala index 50d6ba1e6f4e6..080aa1d5b9cb1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala @@ -17,19 +17,17 @@ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.catalyst.{CurrentUserContext, InternalRow} -import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, SchemaUnsupported, ViewAlreadyExistsException, ViewSchemaMode} +import org.apache.spark.sql.catalyst.CurrentUserContext +import org.apache.spark.sql.catalyst.analysis.{SchemaUnsupported, ViewSchemaMode} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.catalog.{DependencyList, Identifier, TableCatalog, TableSummary, ViewCatalog} -import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.execution.command.CommandUtils +import org.apache.spark.sql.connector.catalog.{DependencyList, Identifier, TableSummary, ViewCatalog} /** - * Physical plan node for `CREATE VIEW ... WITH METRICS` on a v2 [[ViewCatalog]]. Mirrors - * [[CreateV2ViewExec]]'s flag handling and cross-type collision decoding via the shared - * [[V2ViewPreparation]] trait, and adds metric-view-specific bits (typed - * [[DependencyList]] view-dependencies and `PROP_TABLE_TYPE = METRIC_VIEW`) through the - * trait's optional hooks. + * Physical plan node for `CREATE VIEW ... WITH METRICS` on a v2 [[ViewCatalog]]. Inherits the + * shared CREATE-side `run()` (viewExists short-circuit, OR REPLACE, cross-type collision + * decoding) from [[V2CreateViewPreparation]]; only supplies the metric-view-specific bits + * (no collation, schema-mode UNSUPPORTED, typed view dependencies, `PROP_TABLE_TYPE = + * METRIC_VIEW`) via the [[V2ViewPreparation]] hooks. * * Routed by [[DataSourceV2Strategy]] from * [[org.apache.spark.sql.execution.command.CreateMetricViewCommand]] when the resolved @@ -45,7 +43,7 @@ case class CreateV2MetricViewExec( query: LogicalPlan, allowExisting: Boolean, replace: Boolean, - deps: Option[DependencyList]) extends V2ViewPreparation { + deps: Option[DependencyList]) extends V2CreateViewPreparation { // Metric views don't carry a default-collation override. override def collation: Option[String] = None @@ -62,42 +60,4 @@ case class CreateV2MetricViewExec( override protected def tableType: Option[String] = Some(TableSummary.METRIC_VIEW_TABLE_TYPE) - - override protected def run(): Seq[InternalRow] = { - // CREATE VIEW IF NOT EXISTS short-circuit, identical to CreateV2ViewExec: skip the - // `buildViewInfo` work when a view already sits at the ident. The mixed-catalog - // "table at ident" no-op is handled in the catch block below. - if (allowExisting && catalog.viewExists(identifier)) return Seq.empty - - val info = buildViewInfo() - try { - if (replace) { - CommandUtils.uncacheTableOrView(session, ResolvedIdentifier(catalog, identifier)) - catalog.createOrReplaceView(identifier, info) - } else { - catalog.createView(identifier, info) - } - } catch { - case _: ViewAlreadyExistsException => - // Catalog refused: decode whether a table sits at the ident (cross-type collision) - // or a view (race for plain CREATE / OR REPLACE), and emit the precise error -- or - // no-op for IF NOT EXISTS. Same shape as CreateV2ViewExec. - val isTable = catalog match { - case tc: TableCatalog => tc.tableExists(identifier) - case _ => false - } - if (isTable) { - if (!allowExisting) { - throw QueryCompilationErrors.unsupportedCreateOrReplaceViewOnTableError( - fullNameParts, replace) - } - // CREATE VIEW IF NOT EXISTS over a table is a no-op (v1 parity). - } else if (!allowExisting) { - throw viewAlreadyExists() - } - // else: a view appeared between our viewExists probe and createView; IF NOT EXISTS - // semantics make this a no-op. - } - Seq.empty - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index f046d93219f30..7852ce9c41654 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -57,19 +57,10 @@ private[v2] trait V2ViewPreparation extends LeafV2CommandExec { protected lazy val fullNameParts: Seq[String] = (catalog.name() +: identifier.asMultipartIdentifier).toSeq - /** - * Optional structured dependency list to stamp on the built `ViewInfo`. Plain views leave - * this `None`; metric views populate it with the source-table lineage produced by - * `MetricViewHelper.collectTableDependencies`. - */ + /** Optional structured dependency list to stamp on the built `ViewInfo`. */ protected def viewDependencies: Option[DependencyList] = None - /** - * Optional view sub-kind to stamp on `PROP_TABLE_TYPE`. Plain views leave this `None` so - * the `ViewInfo` constructor's `putIfAbsent` defaults it to `VIEW`; metric views set it to - * `TableSummary.METRIC_VIEW_TABLE_TYPE` so consumers (e.g. `V1Table.toCatalogTable`) can - * round-trip the discriminator. - */ + /** Optional view sub-kind to stamp on `PROP_TABLE_TYPE`; defaults to `VIEW` when `None`. */ protected def tableType: Option[String] = None override def output: Seq[Attribute] = Seq.empty @@ -131,27 +122,19 @@ private[v2] trait V2ViewPreparation extends LeafV2CommandExec { } /** - * Physical plan node for CREATE VIEW on a v2 [[ViewCatalog]]. Dispatches to - * [[ViewCatalog#createView]] for plain CREATE, [[ViewCatalog#createOrReplaceView]] for - * `OR REPLACE`, and short-circuits `IF NOT EXISTS` early via [[ViewCatalog#viewExists]] so - * the view body isn't analyzed when the view already exists. + * Shared CREATE-side `run()` for v2 view-create execs. Adds the `IF NOT EXISTS` short-circuit + * via [[ViewCatalog#viewExists]], dispatches `OR REPLACE` to + * [[ViewCatalog#createOrReplaceView]] vs. plain CREATE to [[ViewCatalog#createView]], and + * decodes `ViewAlreadyExistsException` into the dedicated cross-type collision error when a + * non-view table sits at the ident in a mixed catalog. Subclasses supply only the + * view-shape-specific fields (`allowExisting`, `replace`, plus the [[V2ViewPreparation]] hooks + * such as `viewDependencies` / `tableType`) and inherit `run()` unchanged. */ -case class CreateV2ViewExec( - catalog: ViewCatalog, - identifier: Identifier, - userSpecifiedColumns: Seq[(String, Option[String])], - comment: Option[String], - collation: Option[String], - userProperties: Map[String, String], - originalText: String, - query: LogicalPlan, - allowExisting: Boolean, - replace: Boolean, - viewSchemaMode: ViewSchemaMode) extends V2ViewPreparation { +private[v2] trait V2CreateViewPreparation extends V2ViewPreparation { + def allowExisting: Boolean + def replace: Boolean - override def owner: Option[String] = Some(CurrentUserContext.getCurrentUser) - - override protected def run(): Seq[InternalRow] = { + override final protected def run(): Seq[InternalRow] = { // CREATE VIEW IF NOT EXISTS: short-circuit before `buildViewInfo` if a view already sits // at the ident -- avoids `aliasPlan` / config capture for the common no-op case (matches // v1 `CreateViewCommand.run`). The mixed-catalog "table at ident" no-op is handled in the @@ -190,3 +173,25 @@ case class CreateV2ViewExec( Seq.empty } } + +/** + * Physical plan node for CREATE VIEW on a v2 [[ViewCatalog]]. Inherits the create-side + * `run()` (viewExists short-circuit + OR REPLACE + cross-type decoding) from + * [[V2CreateViewPreparation]]; only supplies the case-class fields and stamps the current + * user as owner. + */ +case class CreateV2ViewExec( + catalog: ViewCatalog, + identifier: Identifier, + userSpecifiedColumns: Seq[(String, Option[String])], + comment: Option[String], + collation: Option[String], + userProperties: Map[String, String], + originalText: String, + query: LogicalPlan, + allowExisting: Boolean, + replace: Boolean, + viewSchemaMode: ViewSchemaMode) extends V2CreateViewPreparation { + + override def owner: Option[String] = Some(CurrentUserContext.getCurrentUser) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 4e0e766289980..264247fea529a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -117,6 +117,11 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { info } + // ============================================================ + // Section 1: CREATE-related tests + // ============================================================ + + test("V2 catalog receives METRIC_VIEW table type and view text via ViewInfo") { withTestCatalogTables { val metricView = MetricView( @@ -172,29 +177,6 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } - test("DROP VIEW succeeds on a V2 metric view") { - withTestCatalogTables { - val metricView = MetricView( - "0.1", - AssetSource(fullSourceTableName), - where = None, - select = metricViewColumns) - createMetricView(fullMetricViewName, metricView) - val ident = Identifier.of(Array(testNamespace), metricViewName) - - assert(MetricViewRecordingCatalog.capturedViews.containsKey(ident)) - - sql(s"DROP VIEW $fullMetricViewName") - assert(!MetricViewRecordingCatalog.capturedViews.containsKey(ident)) - } - } - - test("DROP VIEW IF EXISTS on a non-existent V2 metric view is a no-op") { - withTestCatalogTables { - sql(s"DROP VIEW IF EXISTS $testCatalogName.$testNamespace.does_not_exist") - } - } - test("V2 catalog path captures SQL source and comment") { withTestCatalogTables { val metricView = MetricView( @@ -288,86 +270,6 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } - test("dependency extraction: SQL source JOIN captures both tables") { - withTestCatalogTables { - val secondSource = s"$testCatalogName.$testNamespace.customers" - sql( - s"""CREATE TABLE $secondSource (id INT, name STRING) - |USING foo""".stripMargin) - try { - val joinSql = - s"SELECT c.name, t.count FROM $fullSourceTableName t " + - s"JOIN $secondSource c ON t.count = c.id" - val metricView = MetricView( - "0.1", - SQLSource(joinSql), - where = None, - select = Seq( - Column("name", DimensionExpression("name"), 0), - Column("count_sum", MeasureExpression("sum(count)"), 1))) - createMetricView(fullMetricViewName, metricView) - - val deps = capturedViewInfo().viewDependencies() - assert(deps != null) - val depParts = - deps.dependencies().map(_.asInstanceOf[TableDependency].nameParts().toSeq).toSet - assert(depParts === Set( - Seq(testCatalogName, testNamespace, sourceTableName), - Seq(testCatalogName, testNamespace, "customers")), - s"Expected dependencies on both source tables, got $depParts") - } finally { - sql(s"DROP TABLE IF EXISTS $secondSource") - } - } - } - - test("dependency extraction: SQL source subquery deduplicates same-table references") { - withTestCatalogTables { - val subquerySql = - s"SELECT * FROM $fullSourceTableName " + - s"WHERE count > (SELECT avg(count) FROM $fullSourceTableName)" - val metricView = MetricView( - "0.1", - SQLSource(subquerySql), - where = None, - select = metricViewColumns) - createMetricView(fullMetricViewName, metricView) - - val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().length === 1, - s"Expected 1 deduplicated dependency, got " + - s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") - val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().toSeq === - Seq(testCatalogName, testNamespace, sourceTableName)) - } - } - - test("dependency extraction: SQL source self-join deduplicates same-table references") { - withTestCatalogTables { - val selfJoinSql = - s"SELECT a.region AS a_region, a.count AS a_count " + - s"FROM $fullSourceTableName a JOIN $fullSourceTableName b " + - s"ON a.region = b.region" - val metricView = MetricView( - "0.1", - SQLSource(selfJoinSql), - where = None, - select = Seq( - Column("region", DimensionExpression("a_region"), 0), - Column("count_sum", MeasureExpression("sum(a_count)"), 1))) - createMetricView(fullMetricViewName, metricView) - - val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().length === 1, - s"Expected 1 deduplicated dependency for self-join, got " + - s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") - val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().toSeq === - Seq(testCatalogName, testNamespace, sourceTableName)) - } - } - test("CREATE OR REPLACE VIEW ... WITH METRICS replaces an existing v2 metric view") { withTestCatalogTables { val first = MetricView( @@ -376,7 +278,6 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { where = Some("count > 0"), select = metricViewColumns) createMetricView(fullMetricViewName, first) - val firstYaml = capturedViewInfo().queryText() // Replace with a new body (different WHERE clause). val replacement = MetricView( @@ -395,10 +296,14 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { |$$$$""".stripMargin) val finalInfo = capturedViewInfo() - assert(finalInfo.queryText() === replacementYaml, - "OR REPLACE should swap the captured ViewInfo's queryText.") - assert(finalInfo.queryText() !== firstYaml, - "OR REPLACE should not leave the original captured queryText in place.") + // Assert on the distinguishing fields of the replacement, not on diff vs. the original. + assert(finalInfo.queryText() === replacementYaml) + assert(finalInfo.properties().get(MetricView.PROP_WHERE) === "count > 100") + val deps = finalInfo.viewDependencies() + assert(deps != null && deps.dependencies().length === 1) + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === + Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -493,6 +398,111 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } + test("CREATE VIEW ... WITH METRICS on a non-ViewCatalog catalog fails with " + + "MISSING_CATALOG_ABILITY.VIEWS") { + val ex = intercept[AnalysisException] { + sql( + s"""CREATE VIEW ${MetricViewV2CatalogSuite.noViewCatalogName}.default.mv + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |${MetricViewFactory.toYAML(MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns))} + |$$$$""".stripMargin) + } + assert(ex.getCondition === "MISSING_CATALOG_ABILITY") + assert(ex.getMessage.contains("VIEWS")) + } + + // ============================================================ + // Section 2: Dependency extraction + // ============================================================ + + + test("dependency extraction: SQL source JOIN captures both tables") { + withTestCatalogTables { + val secondSource = s"$testCatalogName.$testNamespace.customers" + sql( + s"""CREATE TABLE $secondSource (id INT, name STRING) + |USING foo""".stripMargin) + try { + val joinSql = + s"SELECT c.name, t.count FROM $fullSourceTableName t " + + s"JOIN $secondSource c ON t.count = c.id" + val metricView = MetricView( + "0.1", + SQLSource(joinSql), + where = None, + select = Seq( + Column("name", DimensionExpression("name"), 0), + Column("count_sum", MeasureExpression("sum(count)"), 1))) + createMetricView(fullMetricViewName, metricView) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null) + val depParts = + deps.dependencies().map(_.asInstanceOf[TableDependency].nameParts().toSeq).toSet + assert(depParts === Set( + Seq(testCatalogName, testNamespace, sourceTableName), + Seq(testCatalogName, testNamespace, "customers")), + s"Expected dependencies on both source tables, got $depParts") + } finally { + sql(s"DROP TABLE IF EXISTS $secondSource") + } + } + } + + test("dependency extraction: SQL source subquery deduplicates same-table references") { + withTestCatalogTables { + val subquerySql = + s"SELECT * FROM $fullSourceTableName " + + s"WHERE count > (SELECT avg(count) FROM $fullSourceTableName)" + val metricView = MetricView( + "0.1", + SQLSource(subquerySql), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, metricView) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null && deps.dependencies().length === 1, + s"Expected 1 deduplicated dependency, got " + + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === + Seq(testCatalogName, testNamespace, sourceTableName)) + } + } + + test("dependency extraction: SQL source self-join deduplicates same-table references") { + withTestCatalogTables { + val selfJoinSql = + s"SELECT a.region AS a_region, a.count AS a_count " + + s"FROM $fullSourceTableName a JOIN $fullSourceTableName b " + + s"ON a.region = b.region" + val metricView = MetricView( + "0.1", + SQLSource(selfJoinSql), + where = None, + select = Seq( + Column("region", DimensionExpression("a_region"), 0), + Column("count_sum", MeasureExpression("sum(a_count)"), 1))) + createMetricView(fullMetricViewName, metricView) + + val deps = capturedViewInfo().viewDependencies() + assert(deps != null && deps.dependencies().length === 1, + s"Expected 1 deduplicated dependency for self-join, got " + + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === + Seq(testCatalogName, testNamespace, sourceTableName)) + } + } + test("dependency extraction: V1 session-catalog source emits 3-part nameParts") { val v1Source = "metric_view_v2_v1source" spark.range(0, 5).toDF("v") @@ -556,6 +566,37 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } + // ============================================================ + // Section 3: SELECT cases + // ============================================================ + + + test("SELECT from a v2 metric view returns aggregated rows") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + // The fixture's `events` source has rows ("region_1", 1, 5.0), ("region_2", 2, 10.0). + // The metric view aggregates by `region` summing `count`. Resolution flows through + // loadRelation -> MetadataOnlyTable(ViewInfo) -> V1Table.toCatalogTable(ViewInfo) -> + // CatalogTableType.METRIC_VIEW -> ResolveMetricView, then the analyzer rewrites the + // view body into Aggregate(Seq(region), Seq(sum(count) AS count_sum)) over `events`. + val rows = sql(s"SELECT region, count_sum FROM $fullMetricViewName ORDER BY region") + .collect() + assert(rows.length === 2) + assert(rows(0).getString(0) === "region_1" && rows(0).getLong(1) === 1L) + assert(rows(1).getString(0) === "region_2" && rows(1).getLong(1) === 2L) + } + } + + // ============================================================ + // Section 4: DESCRIBE cases + // ============================================================ + + test("DESCRIBE TABLE EXTENDED on a v2 metric view round-trips through loadRelation") { withTestCatalogTables { val mv = MetricView( @@ -583,23 +624,82 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } - test("CREATE VIEW ... WITH METRICS on a non-ViewCatalog catalog fails with " + - val ex = intercept[AnalysisException] { - sql( - s"""CREATE VIEW ${MetricViewV2CatalogSuite.noViewCatalogName}.default.mv - |WITH METRICS - |LANGUAGE YAML - |AS - |$$$$ - |${MetricViewFactory.toYAML(MetricView( - "0.1", - AssetSource(fullSourceTableName), - where = None, - select = metricViewColumns))} - |$$$$""".stripMargin) + test("DESCRIBE TABLE on a v2 metric view returns the aliased columns") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + val rows = sql(s"DESCRIBE TABLE $fullMetricViewName").collect() + val byName = rows.map(r => r.getString(0) -> r.getString(1)).toMap + assert(byName.contains("region"), s"Missing 'region' col, got: ${byName.keys}") + assert(byName.contains("count_sum"), s"Missing 'count_sum' col, got: ${byName.keys}") + } + } + + // ============================================================ + // Section 5: DROP / SHOW cases + // ============================================================ + + + test("DROP VIEW succeeds on a V2 metric view") { + withTestCatalogTables { + val metricView = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, metricView) + val ident = Identifier.of(Array(testNamespace), metricViewName) + + assert(MetricViewRecordingCatalog.capturedViews.containsKey(ident)) + + sql(s"DROP VIEW $fullMetricViewName") + assert(!MetricViewRecordingCatalog.capturedViews.containsKey(ident)) + } + } + + test("DROP VIEW IF EXISTS on a non-existent V2 metric view is a no-op") { + withTestCatalogTables { + sql(s"DROP VIEW IF EXISTS $testCatalogName.$testNamespace.does_not_exist") + } + } + + test("SHOW TABLES does not list v2 metric views") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + val tables = sql(s"SHOW TABLES IN $testCatalogName.$testNamespace") + .collect().map(_.getString(1)).toSet + // The fixture's `events` source table is a regular table, so SHOW TABLES sees it. + assert(tables.contains(sourceTableName), + s"SHOW TABLES should list the source table, got: $tables") + // Per the RelationCatalog contract, SHOW TABLES returns tables only -- metric views + // belong on SHOW VIEWS instead. + assert(!tables.contains(metricViewName), + s"SHOW TABLES should not list metric views, got: $tables") + } + } + + test("SHOW VIEWS lists v2 metric views") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + val views = sql(s"SHOW VIEWS IN $testCatalogName.$testNamespace") + .collect().map(_.getString(1)).toSet + assert(views.contains(metricViewName), + s"SHOW VIEWS should list metric views, got: $views") } - assert(ex.getCondition === "MISSING_CATALOG_ABILITY") - assert(ex.getMessage.contains("VIEWS")) } } From fcdc19f8fae15008968835d2efcada9795c4ea56 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Wed, 29 Apr 2026 20:53:45 +0000 Subject: [PATCH 04/17] [SPARK-XXXXX][SQL][FOLLOWUP] Expand SELECT / DROP TABLE / SHOW CREATE TABLE coverage Add the test cases requested on PR #55487: SELECT read-back (section 3) -- 4 new tests modeled on `MetricViewSuite` patterns: - Existing test fixed to use `measure(count_sum)` (the v1 suite shows the required syntax for measure columns) and switched to `checkAnswer` against an equivalent raw aggregation over the source. - `SELECT measure(...) with a WHERE clause on a dimension` -- exercises a filter at the query layer. - `SELECT against a v2 metric view honors the view's pre-defined where clause` -- creates the metric view with `where = Some("count > 1")`, asserts only the matching rows surface. - `SELECT from a v2 metric view supports multiple measures with different aggregations` -- adds price_sum / price_max alongside count_sum. - `SELECT from a v2 metric view supports ORDER BY and LIMIT on measures`. DROP TABLE on a v2 metric view (section 5) -- 2 new tests: - `DROP TABLE on a v2 metric view throws EXPECT_TABLE_NOT_VIEW` -- `DropTableExec`'s `tableExists` probe returns false (RelationCatalog passive filtering), the `viewExists` fallback returns true, so the exec emits `EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE`. Also asserts the metric view is still present in the catalog after the failed DROP. - `DROP TABLE IF EXISTS on a v2 metric view also throws EXPECT_TABLE_NOT_VIEW` -- IF EXISTS does not silence the wrong-type error (v1 parity). SHOW CREATE TABLE on a v2 metric view (section 5) -- 1 new test: - `SHOW CREATE TABLE on a v2 metric view is unsupported` -- routes through `DataSourceV2Strategy`'s `ResolvedPersistentView` case and throws `UNSUPPORTED_FEATURE.TABLE_OPERATION` with operation "SHOW CREATE TABLE". --- .../execution/MetricViewV2CatalogSuite.scala | 160 +++++++++++++++++- 1 file changed, 152 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 264247fea529a..1c13052d42009 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -571,7 +571,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // ============================================================ - test("SELECT from a v2 metric view returns aggregated rows") { + test("SELECT measure(...) from a v2 metric view returns aggregated rows") { withTestCatalogTables { val mv = MetricView( "0.1", @@ -582,13 +582,87 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // The fixture's `events` source has rows ("region_1", 1, 5.0), ("region_2", 2, 10.0). // The metric view aggregates by `region` summing `count`. Resolution flows through // loadRelation -> MetadataOnlyTable(ViewInfo) -> V1Table.toCatalogTable(ViewInfo) -> - // CatalogTableType.METRIC_VIEW -> ResolveMetricView, then the analyzer rewrites the - // view body into Aggregate(Seq(region), Seq(sum(count) AS count_sum)) over `events`. - val rows = sql(s"SELECT region, count_sum FROM $fullMetricViewName ORDER BY region") - .collect() - assert(rows.length === 2) - assert(rows(0).getString(0) === "region_1" && rows(0).getLong(1) === 1L) - assert(rows(1).getString(0) === "region_2" && rows(1).getLong(1) === 2L) + // CatalogTableType.METRIC_VIEW -> ResolveMetricView, which rewrites the view body + // into Aggregate(Seq(region), Seq(sum(count) AS count_sum)) over `events`. The + // `measure(...)` wrapper is required for measure columns -- selecting `count_sum` + // bare would fail (mirrors the v1 `MetricViewSuite` query syntax). + checkAnswer( + sql(s"SELECT region, measure(count_sum) FROM $fullMetricViewName " + + "GROUP BY region ORDER BY region"), + sql(s"SELECT region, sum(count) FROM $fullSourceTableName " + + "GROUP BY region ORDER BY region")) + } + } + + test("SELECT measure(...) with a WHERE clause on a dimension") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + // Filter at the query layer (not on the metric view's own `where:`). + checkAnswer( + sql(s"SELECT measure(count_sum) FROM $fullMetricViewName " + + "WHERE region = 'region_2'"), + sql(s"SELECT sum(count) FROM $fullSourceTableName " + + "WHERE region = 'region_2'")) + } + } + + test("SELECT against a v2 metric view honors the view's pre-defined where clause") { + withTestCatalogTables { + // Pre-define a filter on the metric view itself: only rows with count > 1 should be + // visible to consumers (i.e. region_2 only). + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = Some("count > 1"), + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + checkAnswer( + sql(s"SELECT region, measure(count_sum) FROM $fullMetricViewName " + + "GROUP BY region ORDER BY region"), + sql(s"SELECT region, sum(count) FROM $fullSourceTableName " + + "WHERE count > 1 GROUP BY region ORDER BY region")) + } + } + + test("SELECT from a v2 metric view supports multiple measures with different aggregations") { + withTestCatalogTables { + // Add a second measure (sum of price) so we exercise the multi-measure rewrite path. + val cols = Seq( + Column("region", DimensionExpression("region"), 0), + Column("count_sum", MeasureExpression("sum(count)"), 1), + Column("price_sum", MeasureExpression("sum(price)"), 2), + Column("price_max", MeasureExpression("max(price)"), 3)) + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = cols) + createMetricView(fullMetricViewName, mv) + checkAnswer( + sql(s"SELECT measure(count_sum), measure(price_sum), measure(price_max) " + + s"FROM $fullMetricViewName"), + sql(s"SELECT sum(count), sum(price), max(price) FROM $fullSourceTableName")) + } + } + + test("SELECT from a v2 metric view supports ORDER BY and LIMIT on measures") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + checkAnswer( + sql(s"SELECT region, measure(count_sum) FROM $fullMetricViewName " + + "GROUP BY region ORDER BY 2 DESC LIMIT 1"), + sql(s"SELECT region, sum(count) FROM $fullSourceTableName " + + "GROUP BY region ORDER BY 2 DESC LIMIT 1")) } } @@ -661,6 +735,76 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } + test("DROP TABLE on a v2 metric view throws EXPECT_TABLE_NOT_VIEW") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + + // `DropTableExec` first probes `tableExists` (false for a view per the RelationCatalog + // passive-filtering contract), then falls back to `viewExists` and -- when the entity + // exists as a view but a table was requested -- throws `EXPECT_TABLE_NOT_VIEW` to + // distinguish "wrong type" from "missing". + val ex = intercept[AnalysisException] { + sql(s"DROP TABLE $fullMetricViewName") + } + assert(ex.getCondition === "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + s"Expected EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE, got ${ex.getCondition}: ${ex.getMessage}") + + // The metric view is still present after the failed DROP TABLE. + val ident = Identifier.of(Array(testNamespace), metricViewName) + assert(MetricViewRecordingCatalog.capturedViews.containsKey(ident), + "DROP TABLE on a metric view must not delete it.") + } + } + + test("DROP TABLE IF EXISTS on a v2 metric view also throws EXPECT_TABLE_NOT_VIEW") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + + // IF EXISTS does not silence the wrong-type error: the entity exists, just not as a + // table. (This mirrors the v1 `DropTableCommand` behavior, where `IF EXISTS` only + // short-circuits the not-found branch.) + val ex = intercept[AnalysisException] { + sql(s"DROP TABLE IF EXISTS $fullMetricViewName") + } + assert(ex.getCondition === "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + s"Expected EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE, got ${ex.getCondition}: ${ex.getMessage}") + } + } + + test("SHOW CREATE TABLE on a v2 metric view is unsupported") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + + // SHOW CREATE TABLE on a v2 view (including metric views) is rejected by + // DataSourceV2Strategy via `unsupportedTableOperationError(...)`. There's no + // round-trippable `CREATE VIEW ... WITH METRICS` form yet, so explicit "unsupported" + // is the right answer rather than emitting a misleading plain `CREATE VIEW ...`. + val ex = intercept[AnalysisException] { + sql(s"SHOW CREATE TABLE $fullMetricViewName") + } + assert(ex.getCondition === "UNSUPPORTED_FEATURE.TABLE_OPERATION", + s"Expected UNSUPPORTED_FEATURE.TABLE_OPERATION, got " + + s"${ex.getCondition}: ${ex.getMessage}") + assert(ex.getMessage.contains("SHOW CREATE TABLE"), + s"Error message should mention 'SHOW CREATE TABLE', got: ${ex.getMessage}") + } + } + test("DROP VIEW IF EXISTS on a non-existent V2 metric view is a no-op") { withTestCatalogTables { sql(s"DROP VIEW IF EXISTS $testCatalogName.$testNamespace.does_not_exist") From aeee6a7e85ad8ba49f7be07c46bb0a59029b5f42 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Thu, 30 Apr 2026 02:59:48 +0000 Subject: [PATCH 05/17] [SPARK-XXX][SQL] Fix CI failures: METRIC_VIEW table-type mapping, recursion, scalastyle Address four CI failures introduced by the v2 metric-view changes on this PR: 1. `SparkOperation.tableTypeString` (hive-thriftserver) did not know how to map the new `CatalogTableType.METRIC_VIEW` to a Hive JDBC table type, causing `SparkMetadataOperationSuite` `GET_TABLE_TYPES` to fail with "Unknown table type is found: CatalogTableType(METRIC_VIEW)". Map `METRIC_VIEW` to `"VIEW"` on the JDBC wire (Hive only knows `TABLE` / `VIEW`). 2. `HiveClientImpl.toHiveTableType` (hive) had the same gap, breaking 19 `HiveMetricViewSuite` cases that go through HMS persistence on the V1 session-catalog path. Map `METRIC_VIEW` to `HiveTableType.VIRTUAL_VIEW`. 3. `MetricViewRecordingCatalog.loadRelation` in `MetricViewV2CatalogSuite` recursed infinitely. `class MetricViewRecordingCatalog extends InMemoryTableCatalog with RelationCatalog` linearizes `RelationCatalog` ahead of `InMemoryTableCatalog`, so `super.loadTable` resolved to `RelationCatalog.loadTable`, whose default delegates back to `loadRelation`. Qualify the super call to `super[InMemoryTableCatalog]` to dispatch directly to the table-catalog implementation. 4. Scalastyle import-order violation in `DataSourceV2Strategy.scala`: `metricview.serde.MetricViewFactory` was placed before `execution.datasources.*`. Reorder to satisfy the Scalastyle import-order rule. Signed-off-by: chen.wang --- .../apache/spark/sql/execution/MetricViewV2CatalogSuite.scala | 4 +++- .../apache/spark/sql/hive/thriftserver/SparkOperation.scala | 4 ++-- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 1c13052d42009..ed5a29b682e77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -911,7 +911,9 @@ class MetricViewRecordingCatalog extends InMemoryTableCatalog with RelationCatal val key = (ident.namespace().toSeq, ident.name()) Option(views.get(key)) match { case Some(info) => new MetadataOnlyTable(info, ident.toString) - case None => super.loadTable(ident) // delegate to InMemoryTableCatalog for tables + // Bypass `RelationCatalog.loadTable` (whose default delegates back to `loadRelation`) + // and call `InMemoryTableCatalog.loadTable` directly to avoid infinite recursion. + case None => super[InMemoryTableCatalog].loadTable(ident) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index fea878bcf05d5..10f520314865d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.LogKeys.{HIVE_OPERATION_TYPE, STATEMENT_ID} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.CurrentUserContext.CURRENT_USER import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, SessionCatalog} -import org.apache.spark.sql.catalyst.catalog.CatalogTableType.{EXTERNAL, MANAGED, VIEW} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType.{EXTERNAL, MANAGED, METRIC_VIEW, VIEW} import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.util.Utils @@ -107,7 +107,7 @@ private[hive] trait SparkOperation extends Operation with Logging { def tableTypeString(tableType: CatalogTableType): String = tableType match { case EXTERNAL | MANAGED => "TABLE" - case VIEW => "VIEW" + case VIEW | METRIC_VIEW => "VIEW" case t => throw new IllegalArgumentException(s"Unknown table type is found: $t") } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 898469221796b..f376bc87841de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -1163,7 +1163,7 @@ private[hive] object HiveClientImpl extends Logging { catalogTableType match { case CatalogTableType.EXTERNAL => HiveTableType.EXTERNAL_TABLE case CatalogTableType.MANAGED => HiveTableType.MANAGED_TABLE - case CatalogTableType.VIEW => HiveTableType.VIRTUAL_VIEW + case CatalogTableType.VIEW | CatalogTableType.METRIC_VIEW => HiveTableType.VIRTUAL_VIEW case t => throw new IllegalArgumentException( s"Unknown table type is found at toHiveTableType: $t") From 27f43688a994f27a91c66efd9ce20a7d690d8690 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Thu, 30 Apr 2026 17:36:15 +0000 Subject: [PATCH 06/17] [SPARK-XXX][SQL] Rename test catalog to TableViewCatalog / MetadataTable / loadTableOrView Adapt `MetricViewV2CatalogSuite` to the renames in SPARK-56655, which closed out the still-`Evolving` v2 view API surface introduced by SPARK-52729: - `RelationCatalog` -> `TableViewCatalog` - `MetadataOnlyTable` -> `MetadataTable` - `loadRelation` -> `loadTableOrView` - `ViewCatalog` gained a new abstract `renameView(oldIdent, newIdent)` Updates: - Imports: drop `MetadataOnlyTable`, `RelationCatalog`; add `MetadataTable`, `TableViewCatalog`. - `MetricViewRecordingCatalog`: mix in `TableViewCatalog` instead of `RelationCatalog`; rename the override of `loadRelation` -> `loadTableOrView`, and `MetadataOnlyTable` -> `MetadataTable`. The qualified `super[InMemoryTableCatalog].loadTable(ident)` recursion-bypass continues to work (the default `loadTable` derived from `TableViewCatalog` still delegates to `loadTableOrView`, hence the bypass is still required). - Add a `renameView(oldIdent, newIdent)` implementation: moves the `views` map entry, the `MetricViewRecordingCatalog.capturedViews` entry, and rejects via `ViewAlreadyExistsException` / `NoSuchViewException` per the `TableViewCatalog` contract. - Update doc comments and a few test names that mentioned the old API. Signed-off-by: chen.wang --- .../execution/MetricViewV2CatalogSuite.scala | 51 ++++++++++++------- 1 file changed, 33 insertions(+), 18 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index ed5a29b682e77..153319fba37b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -23,7 +23,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.analysis.{NoSuchViewException, ViewAlreadyExistsException} -import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog, MetadataOnlyTable, RelationCatalog, Table, TableCatalog, TableDependency, TableSummary, ViewInfo} +import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog, MetadataTable, Table, TableCatalog, TableDependency, TableSummary, TableViewCatalog, ViewInfo} import org.apache.spark.sql.metricview.serde.{AssetSource, Column, Constants, DimensionExpression, MeasureExpression, MetricView, MetricViewFactory, SQLSource} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.Metadata @@ -33,7 +33,7 @@ import org.apache.spark.sql.types.Metadata * non-session V2 catalog. Metric views are persisted through the same [[ViewCatalog]] interface * as plain views; the only marker that distinguishes them is `PROP_TABLE_TYPE = METRIC_VIEW` * plus the typed `viewDependencies` field on [[ViewInfo]]. The recording catalog used here is a - * minimal [[RelationCatalog]] so the same instance can also host the source table referenced by + * minimal [[TableViewCatalog]] so the same instance can also host the source table referenced by * the metric view's YAML. */ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { @@ -539,7 +539,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val multiTable = "events_deep" val multiFull = s"$testCatalogName.${multiNamespace.mkString(".")}.$multiTable" withTestCatalogTables { - // The InMemoryTableCatalog (RelationCatalog mixin) supports multi-level namespaces. + // The InMemoryTableCatalog (TableViewCatalog mixin) supports multi-level namespaces. sql(s"CREATE NAMESPACE IF NOT EXISTS $testCatalogName.${multiNamespace.head}") sql(s"CREATE NAMESPACE IF NOT EXISTS " + s"$testCatalogName.${multiNamespace.mkString(".")}") @@ -581,7 +581,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, mv) // The fixture's `events` source has rows ("region_1", 1, 5.0), ("region_2", 2, 10.0). // The metric view aggregates by `region` summing `count`. Resolution flows through - // loadRelation -> MetadataOnlyTable(ViewInfo) -> V1Table.toCatalogTable(ViewInfo) -> + // loadTableOrView -> MetadataTable(ViewInfo) -> V1Table.toCatalogTable(ViewInfo) -> // CatalogTableType.METRIC_VIEW -> ResolveMetricView, which rewrites the view body // into Aggregate(Seq(region), Seq(sum(count) AS count_sum)) over `events`. The // `measure(...)` wrapper is required for measure columns -- selecting `count_sum` @@ -671,7 +671,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // ============================================================ - test("DESCRIBE TABLE EXTENDED on a v2 metric view round-trips through loadRelation") { + test("DESCRIBE TABLE EXTENDED on a v2 metric view round-trips through loadTableOrView") { withTestCatalogTables { val mv = MetricView( "0.1", @@ -681,8 +681,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val yaml = createMetricView(fullMetricViewName, mv) // DESCRIBE TABLE EXTENDED resolves the ident through `Analyzer.lookupTableOrView`, - // which calls `RelationCatalog.loadRelation` once and gets back a - // `MetadataOnlyTable(ViewInfo)`. `V1Table.toCatalogTable(ViewInfo)` reconstructs the + // which calls `TableViewCatalog.loadTableOrView` once and gets back a + // `MetadataTable(ViewInfo)`. `V1Table.toCatalogTable(ViewInfo)` reconstructs the // V1 representation; the resulting `CatalogTable.toJsonLinkedHashMap` (interface.scala) // then emits view-context rows because `tableType == METRIC_VIEW` was added to the // VIEW gate. Without that gate fix, "View Text" / "View Original Text" disappear. @@ -744,7 +744,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { select = metricViewColumns) createMetricView(fullMetricViewName, mv) - // `DropTableExec` first probes `tableExists` (false for a view per the RelationCatalog + // `DropTableExec` first probes `tableExists` (false for a view per the TableViewCatalog // passive-filtering contract), then falls back to `viewExists` and -- when the entity // exists as a view but a table was requested -- throws `EXPECT_TABLE_NOT_VIEW` to // distinguish "wrong type" from "missing". @@ -824,7 +824,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // The fixture's `events` source table is a regular table, so SHOW TABLES sees it. assert(tables.contains(sourceTableName), s"SHOW TABLES should list the source table, got: $tables") - // Per the RelationCatalog contract, SHOW TABLES returns tables only -- metric views + // Per the TableViewCatalog contract, SHOW TABLES returns tables only -- metric views // belong on SHOW VIEWS instead. assert(!tables.contains(metricViewName), s"SHOW TABLES should not list metric views, got: $tables") @@ -852,7 +852,7 @@ object MetricViewV2CatalogSuite { } /** - * Minimal [[RelationCatalog]] used by [[MetricViewV2CatalogSuite]]. Layers `ViewCatalog` + * Minimal [[TableViewCatalog]] used by [[MetricViewV2CatalogSuite]]. Layers `ViewCatalog` * methods over [[InMemoryTableCatalog]] (which provides table storage + namespace ops) and * captures every [[ViewInfo]] passed to `createView` so tests can inspect the typed payload. * @@ -860,7 +860,7 @@ object MetricViewV2CatalogSuite { * the view identifiers; the source table created by the test fixture is stored separately in * the inherited table catalog. */ -class MetricViewRecordingCatalog extends InMemoryTableCatalog with RelationCatalog { +class MetricViewRecordingCatalog extends InMemoryTableCatalog with TableViewCatalog { private val views = new ConcurrentHashMap[(Seq[String], String), ViewInfo]() @@ -875,9 +875,9 @@ class MetricViewRecordingCatalog extends InMemoryTableCatalog with RelationCatal out.asScala.toArray } - // `loadView`, `tableExists`, and `viewExists` are inherited from `RelationCatalog`'s - // defaults, which derive from `loadRelation` -- a stored `ViewInfo` is wrapped in - // `MetadataOnlyTable` by `loadRelation` and the defaults unwrap it correctly. + // `loadView`, `tableExists`, and `viewExists` are inherited from `TableViewCatalog`'s + // defaults, which derive from `loadTableOrView` -- a stored `ViewInfo` is wrapped in + // `MetadataTable` by `loadTableOrView` and the defaults unwrap it correctly. override def createView(ident: Identifier, info: ViewInfo): ViewInfo = { val key = (ident.namespace().toSeq, ident.name()) @@ -905,13 +905,28 @@ class MetricViewRecordingCatalog extends InMemoryTableCatalog with RelationCatal removed } - // -- RelationCatalog single-RPC perf path -- + override def renameView(oldIdent: Identifier, newIdent: Identifier): Unit = { + val oldKey = (oldIdent.namespace().toSeq, oldIdent.name()) + val newKey = (newIdent.namespace().toSeq, newIdent.name()) + val existing = views.get(oldKey) + if (existing == null) throw new NoSuchViewException(oldIdent) + if (views.putIfAbsent(newKey, existing) != null) { + throw new ViewAlreadyExistsException(newIdent) + } + views.remove(oldKey) + val captured = MetricViewRecordingCatalog.capturedViews.remove(oldIdent) + if (captured != null) { + MetricViewRecordingCatalog.capturedViews.put(newIdent, captured) + } + } + + // -- TableViewCatalog single-RPC perf path -- - override def loadRelation(ident: Identifier): Table = { + override def loadTableOrView(ident: Identifier): Table = { val key = (ident.namespace().toSeq, ident.name()) Option(views.get(key)) match { - case Some(info) => new MetadataOnlyTable(info, ident.toString) - // Bypass `RelationCatalog.loadTable` (whose default delegates back to `loadRelation`) + case Some(info) => new MetadataTable(info, ident.toString) + // Bypass `TableViewCatalog.loadTable` (whose default delegates back to `loadTableOrView`) // and call `InMemoryTableCatalog.loadTable` directly to avoid infinite recursion. case None => super[InMemoryTableCatalog].loadTable(ident) } From a1c292715c0678c06d1b3a15a8ad37235a9a89db Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Thu, 30 Apr 2026 20:48:17 +0000 Subject: [PATCH 07/17] [SPARK-XXX][SQL] Fix MetricViewV2CatalogSuite + HMS metric-view round-trip Five fixes to recover the metric-view test suites after the latest rebase on apache/spark master: 1. `MetricViewV2CatalogSuite.withTestCatalogTables`: a few negative tests pre-create a regular table at `mv` to exercise cross-type collisions, but the previous cleanup only did `DROP VIEW IF EXISTS mv`, leaving the table behind to poison every subsequent test in the suite (cleanup raised `WRONG_COMMAND_FOR_OBJECT_TYPE` because mv was a TABLE, not a VIEW). Sweep `mv` as both view and table in cleanup, wrapped in `Try` so the symmetric "wrong command for type" errors don't escape. 2. Test assertions on `info.queryText() === yaml`: the captured queryText includes the `\n` between the SQL `$$` markers and the YAML body, but `MetricViewFactory.toYAML(...)` doesn't emit surrounding newlines. Trim both sides before the equality check (3 sites). 3. `MISSING_CATALOG_ABILITY` assertion: SPARK-56655 added the `.VIEWS` subclass; the bare error condition no longer surfaces directly. Update the assertion to expect `MISSING_CATALOG_ABILITY.VIEWS`. 4. `MetricViewRecordingCatalog.{createView, replaceView}`: the `TableViewCatalog` active-rejection contract requires `createView` to throw `ViewAlreadyExistsException` (and `replaceView` to throw `NoSuchViewException`) when a *table* sits at the ident, not just when a view does. The previous override only checked `views`, so the pre-table-then-create-view test never raised the cross-type collision `CreateV2MetricViewExec` is supposed to decode into `EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE`. Add a tables-only existence probe (bypassing `TableViewCatalog.tableExists`'s default, which would recurse through our own `loadTableOrView`) and reject up front. 5. `HiveExternalCatalog` HMS round-trip for metric views: `HiveTableType` has no `METRIC_VIEW` variant -- both regular views and metric views serialize as `VIRTUAL_VIEW`, and `HiveClientImpl.getTableOption` always maps `VIRTUAL_VIEW` back to `CatalogTableType.VIEW`, dropping the metric-view distinction on read. Result: `HiveMetricViewSuite` (which reuses the same v1 `MetricViewSuite` against `HiveExternalCatalog`) saw the YAML body parsed as SQL, raising `PARSE_SYNTAX_ERROR` on every test. Persist a `view.subType = METRIC_VIEW` property on write (`tableMetaToTableProps`) and lift `tableType` back to `METRIC_VIEW` on read (`restoreTableMetadata`) when the marker is present. Property name follows the existing `view.*` convention so it round-trips through HMS like other view metadata. Also factored the `metric_view.*` column metadata preservation across user-specified renames into a hookable `retainColumnMetadata: Boolean` on `V2ViewPreparation` (defaults to `false`); `CreateV2MetricViewExec` overrides it to `true`. Mirrors what `ViewHelper.prepareTable(isMetricView = true)` already does on the v1 path, fixing the `key not found: metric_view.type` failure on the v2 "user-specified column names with comments preserve metric_view.* metadata" test. Signed-off-by: chen.wang --- .../sql/catalyst/catalog/interface.scala | 9 ++++ .../v2/CreateV2MetricViewExec.scala | 6 +++ .../datasources/v2/CreateV2ViewExec.scala | 13 +++++- .../execution/MetricViewV2CatalogSuite.scala | 43 ++++++++++++++++--- .../spark/sql/hive/HiveExternalCatalog.scala | 16 +++++++ 5 files changed, 80 insertions(+), 7 deletions(-) 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 afe1406272373..17c2a00279932 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 @@ -742,6 +742,15 @@ object CatalogTable { val VIEW_CATALOG_AND_NAMESPACE = VIEW_PREFIX + "catalogAndNamespace.numParts" val VIEW_CATALOG_AND_NAMESPACE_PART_PREFIX = VIEW_PREFIX + "catalogAndNamespace.part." + /** + * View sub-type marker persisted in `properties` so the metric-view distinction survives a + * round-trip through external catalogs whose enum can't carry it (e.g. the Hive Metastore, + * which only knows `VIRTUAL_VIEW`). When this property is set, the in-memory `tableType` + * upgrades from [[CatalogTableType.VIEW]] back to [[CatalogTableType.METRIC_VIEW]] on read. + */ + val VIEW_SUB_TYPE = VIEW_PREFIX + "subType" + val VIEW_SUB_TYPE_METRIC_VIEW = "METRIC_VIEW" + /** * Check if a CatalogTable is a metric view. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala index 080aa1d5b9cb1..d25239aac949d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2MetricViewExec.scala @@ -60,4 +60,10 @@ case class CreateV2MetricViewExec( override protected def tableType: Option[String] = Some(TableSummary.METRIC_VIEW_TABLE_TYPE) + + // The analyzer attaches `metric_view.type` / `metric_view.expr` keys to each output + // attribute's metadata; `aliasPlan`'s default re-projection drops them when the user + // supplies a column-rename clause. Mirror v1 `ViewHelper.prepareTable(isMetricView = true)` + // by retaining metadata across the rename. + override protected def retainColumnMetadata: Boolean = true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index 7852ce9c41654..4e10c7d3ab284 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -63,6 +63,15 @@ private[v2] trait V2ViewPreparation extends LeafV2CommandExec { /** Optional view sub-kind to stamp on `PROP_TABLE_TYPE`; defaults to `VIEW` when `None`. */ protected def tableType: Option[String] = None + /** + * Whether `aliasPlan` should preserve any column metadata the analyzer attached to the + * source plan when re-aliasing user-specified column names. Plain views default to `false` + * (matches v1 `CreateViewCommand`); metric views override to `true` so the analyzer-injected + * `metric_view.type` / `metric_view.expr` keys survive a `CREATE VIEW (c1, c2, ...)` + * column rename (matches v1 `ViewHelper.prepareTable(isMetricView = true)`). + */ + protected def retainColumnMetadata: Boolean = false + override def output: Seq[Attribute] = Seq.empty protected def buildViewInfo(): ViewInfo = { @@ -85,7 +94,9 @@ private[v2] trait V2ViewPreparation extends LeafV2CommandExec { SchemaUtils.checkIndeterminateCollationInSchema(query.schema) val aliasedSchema = CharVarcharUtils.getRawSchema( - aliasPlan(session, query, userSpecifiedColumns).schema, session.sessionState.conf) + aliasPlan(session, query, userSpecifiedColumns, retainMetadata = retainColumnMetadata) + .schema, + session.sessionState.conf) SchemaUtils.checkColumnNameDuplication( aliasedSchema.fieldNames.toImmutableArraySeq, session.sessionState.conf.resolver) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 153319fba37b2..d869040eedc50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -85,8 +85,17 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { |USING foo AS SELECT * FROM metric_view_v2_source""".stripMargin) body } finally { - sql(s"DROP VIEW IF EXISTS $fullMetricViewName") - sql(s"DROP TABLE IF EXISTS $fullSourceTableName") + // The metric-view ident `mv` may have ended up as either a view (most tests) or as a + // pre-created table (a few negative tests pre-create a table at the same ident to + // exercise cross-type collisions). Sweep both kinds so subsequent tests in the suite + // start from a clean catalog state. Wrap each DROP in a Try because: + // - DROP VIEW IF EXISTS on a leftover *table* throws WRONG_COMMAND_FOR_OBJECT_TYPE + // under master's new DropViewExec active-rejection contract. + // - DROP TABLE IF EXISTS on a leftover *view* throws the symmetric error. + // - On a totally clean state both are silent no-ops. + scala.util.Try(sql(s"DROP VIEW IF EXISTS $fullMetricViewName")) + scala.util.Try(sql(s"DROP TABLE IF EXISTS $fullMetricViewName")) + scala.util.Try(sql(s"DROP TABLE IF EXISTS $fullSourceTableName")) spark.catalog.dropTempView("metric_view_v2_source") MetricViewRecordingCatalog.reset() } @@ -137,7 +146,10 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // back to `CatalogTableType.METRIC_VIEW`. assert(info.properties().get(TableCatalog.PROP_TABLE_TYPE) === TableSummary.METRIC_VIEW_TABLE_TYPE) - assert(info.queryText() === yaml) + // The captured queryText is the raw text between `$$ ... $$` -- including the leading + // and trailing newline our SQL fixture inserts -- so trim before comparing to the + // pre-substitution YAML body. + assert(info.queryText().trim === yaml.trim) val deps = info.viewDependencies() assert(deps != null) @@ -297,7 +309,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val finalInfo = capturedViewInfo() // Assert on the distinguishing fields of the replacement, not on diff vs. the original. - assert(finalInfo.queryText() === replacementYaml) + // queryText keeps the surrounding `\n` from the SQL `$$ ... $$` markers; trim first. + assert(finalInfo.queryText().trim === replacementYaml.trim) assert(finalInfo.properties().get(MetricView.PROP_WHERE) === "count > 100") val deps = finalInfo.viewDependencies() assert(deps != null && deps.dependencies().length === 1) @@ -335,7 +348,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { |$replacementYaml |$$$$""".stripMargin) - assert(capturedViewInfo().queryText() === originalYaml, + assert(capturedViewInfo().queryText().trim === originalYaml.trim, "IF NOT EXISTS over an existing metric view should be a no-op.") } } @@ -414,7 +427,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { select = metricViewColumns))} |$$$$""".stripMargin) } - assert(ex.getCondition === "MISSING_CATALOG_ABILITY") + // SPARK-56655 added the `.VIEWS` subclass; the bare `MISSING_CATALOG_ABILITY` no longer + // surfaces directly for the missing-view-ability case. + assert(ex.getCondition === "MISSING_CATALOG_ABILITY.VIEWS") assert(ex.getMessage.contains("VIEWS")) } @@ -879,7 +894,20 @@ class MetricViewRecordingCatalog extends InMemoryTableCatalog with TableViewCata // defaults, which derive from `loadTableOrView` -- a stored `ViewInfo` is wrapped in // `MetadataTable` by `loadTableOrView` and the defaults unwrap it correctly. + // Bypasses `TableViewCatalog.tableExists` (whose default delegates to `loadTableOrView`, + // which checks our `views` map first); we want a tables-only check here so the cross-type + // collision branches in `createView` / `replaceView` see only "is there a *table* at this + // ident?". + private def tableExistsTablesOnly(ident: Identifier): Boolean = + try { super[InMemoryTableCatalog].loadTable(ident); true } + catch { case _: org.apache.spark.sql.catalyst.analysis.NoSuchTableException => false } + override def createView(ident: Identifier, info: ViewInfo): ViewInfo = { + // TableViewCatalog active-rejection contract: createView must throw + // ViewAlreadyExistsException when *either* a view *or* a table sits at the ident. + if (tableExistsTablesOnly(ident)) { + throw new ViewAlreadyExistsException(ident) + } val key = (ident.namespace().toSeq, ident.name()) if (views.putIfAbsent(key, info) != null) { throw new ViewAlreadyExistsException(ident) @@ -889,6 +917,9 @@ class MetricViewRecordingCatalog extends InMemoryTableCatalog with TableViewCata } override def replaceView(ident: Identifier, info: ViewInfo): ViewInfo = { + // Per the TableViewCatalog contract, replaceView must surface NoSuchViewException + // when a *table* sits at the ident (not silently succeed and shadow the table). + if (tableExistsTablesOnly(ident)) throw new NoSuchViewException(ident) val key = (ident.namespace().toSeq, ident.name()) if (!views.containsKey(key)) throw new NoSuchViewException(ident) views.put(key, info) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index a72fbce74be1c..3f5e8cd266c9f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -452,6 +452,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat val properties = new mutable.HashMap[String, String] properties.put(CREATED_SPARK_VERSION, table.createVersion) + + // Hive's `HiveTableType` enum has no metric-view variant -- it stores both regular views + // and metric views as `VIRTUAL_VIEW`. Persist a property marker so `restoreTableMetadata` + // can lift the round-tripped `CatalogTableType.VIEW` back to `CatalogTableType.METRIC_VIEW`. + if (table.tableType == CatalogTableType.METRIC_VIEW) { + properties.put(CatalogTable.VIEW_SUB_TYPE, CatalogTable.VIEW_SUB_TYPE_METRIC_VIEW) + } // This is for backward compatibility to Spark 2 to read tables with char/varchar created by // Spark 3.1. At read side, we will restore a table schema from its properties. So, we need to // clear the `varchar(n)` and `char(n)` and replace them with `string` as Spark 2 does not have @@ -836,6 +843,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat var table = inputTable + // HMS round-trips both regular views and metric views as `HiveTableType.VIRTUAL_VIEW`, + // which `HiveClientImpl.getTableOption` always maps back to `CatalogTableType.VIEW`. Lift + // it back to `CatalogTableType.METRIC_VIEW` when the persisted sub-type marker is present. + if (table.tableType == VIEW && + table.properties.get(CatalogTable.VIEW_SUB_TYPE) + .contains(CatalogTable.VIEW_SUB_TYPE_METRIC_VIEW)) { + table = table.copy(tableType = METRIC_VIEW) + } + table.properties.get(DATASOURCE_PROVIDER) match { case None if table.tableType == VIEW || table.tableType == METRIC_VIEW => // If this is a view created by Spark 2.2 or higher versions, we should restore its schema From d4474e7aaab6d03b5e108606bdce17ee7178898b Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Thu, 30 Apr 2026 22:44:56 +0000 Subject: [PATCH 08/17] [SPARK-XXXXX][SQL] Public StructField.json / .fromJson + Column.fromStructField Add three small public APIs that let v2 catalog connectors (such as the Unity Catalog Spark connector built on TableViewCatalog) round-trip a Spark StructField through external storage without reaching for private[sql] helpers or the singleton-StructType wrap workaround: - StructField.json / StructField.prettyJson: public counterparts of DataType.json / DataType.prettyJson, exposing the existing private[sql] jsonValue. - StructField.fromJson(String): companion-object parser that mirrors DataType.fromJson and is the inverse of StructField.json. - Column.fromStructField(StructField): static factory in the catalog Column interface that maps a Spark StructField (with metadata) into a connector Column (with metadataInJSON), symmetric to TableInfo.schema() which already goes the other way via CatalogV2Util.v2ColumnsToStructType. Widens DataType.parseStructField from private to private[sql] so the new StructField.fromJson companion can call it directly. Tests: - DataTypeSuite covers StructField round-trip via .json / .fromJson with metric_view-style metadata + comment, and an empty-metadata field. - CatalogV2UtilSuite covers Column.fromStructField for both the metadata-bearing and empty-metadata cases. Signed-off-by: chen.wang --- .../org/apache/spark/sql/types/DataType.scala | 2 +- .../apache/spark/sql/types/StructField.scala | 30 +++++++++++++++++ .../spark/sql/connector/catalog/Column.java | 17 ++++++++++ .../catalog/CatalogV2UtilSuite.scala | 32 ++++++++++++++++++- .../spark/sql/types/DataTypeSuite.scala | 24 ++++++++++++++ 5 files changed, 103 insertions(+), 2 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala index 48a6514440dd3..26df4e8e9a58f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -311,7 +311,7 @@ object DataType { messageParameters = Map("invalidType" -> compact(render(other)))) } - private def parseStructField(json: JValue): StructField = json match { + private[sql] def parseStructField(json: JValue): StructField = json match { case JSortedObject( ("metadata", JObject(metadataFields)), ("name", JString(name)), diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala index eb3d30051880a..3860b3a0f5ce4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.json4s.{JObject, JString} import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, parse, pretty, render} import org.apache.spark.SparkException import org.apache.spark.annotation.Stable @@ -75,6 +76,20 @@ case class StructField( ("metadata" -> metadataJson) } + /** + * The compact JSON representation of this StructField, including name, type, nullable, and + * metadata. Inverse of [[StructField.fromJson]]. + * + * @since 4.2.0 + */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this StructField. + * + * @since 4.2.0 + */ + def prettyJson: String = pretty(render(jsonValue)) + private[sql] def dataTypeJsonValue: JValue = { if (collationMetadata.isEmpty) return dataType.jsonValue @@ -273,3 +288,18 @@ case class StructField( s"$getDDLDefault$getDDLComment" } } + +/** + * @since 4.2.0 + */ +@Stable +object StructField { + + /** + * Parses a JSON string produced by [[StructField.json]] back into a `StructField`. The JSON + * must encode a single field with `name`, `type`, `nullable`, and `metadata`. + * + * @since 4.2.0 + */ + def fromJson(json: String): StructField = DataType.parseStructField(parse(json)) +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java index 537c2edd11285..eb2492cf42796 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java @@ -24,6 +24,8 @@ import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.internal.connector.ColumnImpl; import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; /** * An interface representing a column of a {@link Table}. It defines basic properties of a column, @@ -122,6 +124,21 @@ static Column create( /* id = */ null); } + /** + * Creates a {@link Column} from a Spark {@link StructField}, preserving name, dataType, + * nullable, comment, and the field metadata as a JSON string. Fields with empty metadata + * map to a column with a {@code null} {@link #metadataInJSON()}. + * + * @since 4.2.0 + */ + static Column fromStructField(StructField field) { + String comment = field.getComment().isDefined() ? field.getComment().get() : null; + String metadataJson = field.metadata().equals(Metadata.empty()) + ? null + : field.metadata().json(); + return create(field.name(), field.dataType(), field.nullable(), comment, metadataJson); + } + /** * Returns the name of this table column. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala index eda401ceb6bdf..d24eef8680eac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -21,7 +21,7 @@ import org.mockito.Mockito.{mock, when} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{IntegerType, MetadataBuilder, StringType, StructField} class CatalogV2UtilSuite extends SparkFunSuite { test("Load relation should encode the identifiers for V2Relations") { @@ -37,4 +37,34 @@ class CatalogV2UtilSuite extends SparkFunSuite { assert(v2Relation.catalog.exists(_ == testCatalog)) assert(v2Relation.identifier.exists(_ == ident)) } + + test("Column.fromStructField preserves name, dataType, nullable, comment, and metadata") { + val metadata = new MetadataBuilder() + .putString("metric_view.type", "dimension") + .putString("metric_view.expr", "region") + .build() + val field = StructField("region", StringType, nullable = true, metadata) + .withComment("dim col") + val column = Column.fromStructField(field) + + assert(column.name() == "region") + assert(column.dataType() == StringType) + assert(column.nullable()) + assert(column.comment() == "dim col") + // The metric_view.* keys must survive round-trip via metadataInJSON; comment is also + // present in the metadata under key "comment" by Spark's withComment convention. + val parsedMetadata = org.apache.spark.sql.types.Metadata.fromJson(column.metadataInJSON()) + assert(parsedMetadata.getString("metric_view.type") == "dimension") + assert(parsedMetadata.getString("metric_view.expr") == "region") + } + + test("Column.fromStructField on a field with empty metadata returns null metadataInJSON") { + val field = StructField("c", IntegerType, nullable = false) + val column = Column.fromStructField(field) + assert(column.name() == "c") + assert(column.dataType() == IntegerType) + assert(!column.nullable()) + assert(column.comment() == null) + assert(column.metadataInJSON() == null) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index ce4f5e89be2b8..91dbebfc3ed87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -315,6 +315,30 @@ class DataTypeSuite extends SparkFunSuite { checkDataTypeFromJson(structType) checkDataTypeFromDDL(structType) + test("StructField json round-trip preserves name, type, nullable, comment, metadata") { + val baseMetadata = new MetadataBuilder() + .putString("metric_view.type", "dimension") + .putString("metric_view.expr", "region") + .build() + val field = StructField("region", StringType, nullable = true, baseMetadata) + .withComment("dim col") + val parsed = StructField.fromJson(field.json) + assert(parsed.name == "region") + assert(parsed.dataType == StringType) + assert(parsed.nullable) + assert(parsed.getComment().contains("dim col")) + // `withComment` adds a "comment" key to metadata; it must round-trip alongside the + // metric_view.* keys. + assert(parsed.metadata.getString("metric_view.type") == "dimension") + assert(parsed.metadata.getString("metric_view.expr") == "region") + } + + test("StructField json round-trip with empty metadata and no comment") { + val field = StructField("c", IntegerType, nullable = false) + val parsed = StructField.fromJson(field.json) + assert(parsed == field) + } + test("fromJson throws an exception when given type string is invalid") { checkError( exception = intercept[SparkIllegalArgumentException] { From 0b26f9590bf36967324958ee9998b0cddea8f65a Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Fri, 1 May 2026 15:52:57 +0000 Subject: [PATCH 09/17] [SPARK-XXXXX][SQL][FOLLOWUP] Address cloud-fan round 2 review on PR #55487 Address the four new findings from cloud-fan's round 2 review of PR #55487 plus the corresponding CI fixes. Combines the revert of the StructField/Column public-API commit (per cloud-fan's split request) with the three remaining metric-view fixes into one commit. Cloud-fan finding #1 (`verifyAlterTableType` audit miss): * `ddl.scala` `verifyAlterTableType` (lines 1086-1103) was missed in the `tableType == VIEW` -> `VIEW || METRIC_VIEW` audit. Without this fix, `ALTER TABLE RENAME TO ...` silently succeeds (renames as if a regular table), and `ALTER VIEW RENAME TO ...` throws `cannotAlterTableWithAlterViewError` (wrong category for a view- kind). Widen both arms to also accept `METRIC_VIEW`. Cloud-fan finding #2/#3 (StructField commit split-out): * Revert d55143f5b6 ("Public StructField.json / .fromJson + Column.fromStructField"). None of the three new public APIs are called by code in this PR -- they exist to support the downstream Unity Catalog connector. Per cloud-fan's request, split them out into a separate PR with its own JIRA ticket so they can be reviewed independently. Reverting the commit also removes the `Column.fromStructField` divergence from `structFieldToV2Column` (which would have duplicated the comment field in `metadataInJSON`); the split-out PR fixes that inline. The revert also fixes two CI failures introduced by the same commit: - `mimaReportBinaryIssues` losing `StructField.tupled` / `StructField.curried` / `AbstractFunction4` parent on the `StructField` companion object (sql-api 4.0.0 binary surface). - `Javaunidoc` failing on the `{@link StructField.json}` / `{@link StructField.fromJson}` references that javadoc rejects (member separator must be `#`, not `.`). Cloud-fan finding #4 (multi-level namespace lift): * `MetricViewHelper.analyzeMetricViewText` previously required a `TableIdentifier`, which can only carry up to 3 name parts. v2 metric views with multi-level-namespace targets (e.g. `cat.db1.db2.mv`) failed at `ident.asTableIdentifier` with `requiresSinglePartNamespaceError`. Lift the helper to take `nameParts: Seq[String]` directly. The synthetic `CatalogTable` used as analysis context still needs a TableIdentifier (it's a v1 type); we collapse the intermediate namespace components into the synthetic `database` slot via dot-join. Since the synthetic identifier is never used to resolve the view body itself, this collapse is observationally invisible to the analyzed plan, and `verifyTemporaryObjectsNotExists` still receives the full `nameParts` so error rendering preserves the multi-part form. Update both call sites: - `CreateMetricViewCommand.createMetricViewInSessionCatalog` (v1) passes `name.nameParts`. - `DataSourceV2Strategy` (v2) passes `(catalog.name() +: ident.namespace() :+ ident.name())`. Add a new test in `MetricViewV2CatalogSuite` that creates a metric view at `testcat.ns_a.ns_b.mv_deep` and asserts the create succeeds + the captured `ViewInfo` carries `PROP_TABLE_TYPE = METRIC_VIEW`. Test realignment to SPARK-56655's new contract: * SPARK-56655 (master) added several v2 view DDL behaviors that changed the error categories surfaced for cross-type collisions and added view-aware listing on `TableViewCatalog`. Realign 5 existing test assertions in `MetricViewV2CatalogSuite`: - "CREATE VIEW ... WITH METRICS over a v2 table at the ident": `EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE` -> `TABLE_OR_VIEW_ ALREADY_EXISTS` (master's analyzer-time pre-check fires before the exec-time decoding in `CreateV2MetricViewExec.run`). - "DROP TABLE on a v2 metric view" + "DROP TABLE IF EXISTS on a v2 metric view": `EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE` -> `WRONG_COMMAND_FOR_OBJECT_TYPE` ("Use DROP VIEW instead", raised by master's new `DropTableExec.run`). - "SHOW TABLES does not list v2 metric views" -> "SHOW TABLES on a v2 TableViewCatalog lists both tables and metric views": SPARK-56655 routes SHOW TABLES on a `TableViewCatalog` through `listRelationSummaries` so views appear alongside tables in the output (matches v1 SHOW TABLES on a session catalog). - DESCRIBE TABLE EXTENDED `View Text` row: trim before comparing to the pre-substitution YAML body (the row carries the raw text between the SQL `$$ ... $$` markers, including the surrounding `\n`). * Add an explicit `SHOW CREATE TABLE` rejection guard for v2 metric views in `DataSourceV2Strategy`. Master's new `ShowCreateV2ViewExec` would emit a plain `CREATE VIEW AS `, which is not round-trippable as metric-view DDL (the right form is `CREATE VIEW WITH METRICS LANGUAGE YAML AS $$ $$`). Reject up front with `unsupportedTableOperationError("SHOW CREATE TABLE")` rather than emit lossy DDL; the existing test already expects this exact behavior. Signed-off-by: chen.wang --- .../org/apache/spark/sql/types/DataType.scala | 2 +- .../apache/spark/sql/types/StructField.scala | 30 ----- .../spark/sql/connector/catalog/Column.java | 17 --- .../catalog/CatalogV2UtilSuite.scala | 32 +---- .../spark/sql/types/DataTypeSuite.scala | 24 ---- .../spark/sql/execution/command/ddl.scala | 4 +- .../command/metricViewCommands.scala | 35 +++++- .../datasources/v2/DataSourceV2Strategy.scala | 19 ++- .../v2/V2ViewInspectionExecs.scala | 11 +- .../execution/MetricViewV2CatalogSuite.scala | 110 +++++++++++++----- 10 files changed, 142 insertions(+), 142 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala index 26df4e8e9a58f..48a6514440dd3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -311,7 +311,7 @@ object DataType { messageParameters = Map("invalidType" -> compact(render(other)))) } - private[sql] def parseStructField(json: JValue): StructField = json match { + private def parseStructField(json: JValue): StructField = json match { case JSortedObject( ("metadata", JObject(metadataFields)), ("name", JString(name)), diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala index 3860b3a0f5ce4..eb3d30051880a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import org.json4s.{JObject, JString} import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ -import org.json4s.jackson.JsonMethods.{compact, parse, pretty, render} import org.apache.spark.SparkException import org.apache.spark.annotation.Stable @@ -76,20 +75,6 @@ case class StructField( ("metadata" -> metadataJson) } - /** - * The compact JSON representation of this StructField, including name, type, nullable, and - * metadata. Inverse of [[StructField.fromJson]]. - * - * @since 4.2.0 - */ - def json: String = compact(render(jsonValue)) - - /** The pretty (i.e. indented) JSON representation of this StructField. - * - * @since 4.2.0 - */ - def prettyJson: String = pretty(render(jsonValue)) - private[sql] def dataTypeJsonValue: JValue = { if (collationMetadata.isEmpty) return dataType.jsonValue @@ -288,18 +273,3 @@ case class StructField( s"$getDDLDefault$getDDLComment" } } - -/** - * @since 4.2.0 - */ -@Stable -object StructField { - - /** - * Parses a JSON string produced by [[StructField.json]] back into a `StructField`. The JSON - * must encode a single field with `name`, `type`, `nullable`, and `metadata`. - * - * @since 4.2.0 - */ - def fromJson(json: String): StructField = DataType.parseStructField(parse(json)) -} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java index eb2492cf42796..537c2edd11285 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java @@ -24,8 +24,6 @@ import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.internal.connector.ColumnImpl; import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; /** * An interface representing a column of a {@link Table}. It defines basic properties of a column, @@ -124,21 +122,6 @@ static Column create( /* id = */ null); } - /** - * Creates a {@link Column} from a Spark {@link StructField}, preserving name, dataType, - * nullable, comment, and the field metadata as a JSON string. Fields with empty metadata - * map to a column with a {@code null} {@link #metadataInJSON()}. - * - * @since 4.2.0 - */ - static Column fromStructField(StructField field) { - String comment = field.getComment().isDefined() ? field.getComment().get() : null; - String metadataJson = field.metadata().equals(Metadata.empty()) - ? null - : field.metadata().json(); - return create(field.name(), field.dataType(), field.nullable(), comment, metadataJson); - } - /** * Returns the name of this table column. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala index d24eef8680eac..eda401ceb6bdf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -21,7 +21,7 @@ import org.mockito.Mockito.{mock, when} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.{IntegerType, MetadataBuilder, StringType, StructField} +import org.apache.spark.sql.types.IntegerType class CatalogV2UtilSuite extends SparkFunSuite { test("Load relation should encode the identifiers for V2Relations") { @@ -37,34 +37,4 @@ class CatalogV2UtilSuite extends SparkFunSuite { assert(v2Relation.catalog.exists(_ == testCatalog)) assert(v2Relation.identifier.exists(_ == ident)) } - - test("Column.fromStructField preserves name, dataType, nullable, comment, and metadata") { - val metadata = new MetadataBuilder() - .putString("metric_view.type", "dimension") - .putString("metric_view.expr", "region") - .build() - val field = StructField("region", StringType, nullable = true, metadata) - .withComment("dim col") - val column = Column.fromStructField(field) - - assert(column.name() == "region") - assert(column.dataType() == StringType) - assert(column.nullable()) - assert(column.comment() == "dim col") - // The metric_view.* keys must survive round-trip via metadataInJSON; comment is also - // present in the metadata under key "comment" by Spark's withComment convention. - val parsedMetadata = org.apache.spark.sql.types.Metadata.fromJson(column.metadataInJSON()) - assert(parsedMetadata.getString("metric_view.type") == "dimension") - assert(parsedMetadata.getString("metric_view.expr") == "region") - } - - test("Column.fromStructField on a field with empty metadata returns null metadataInJSON") { - val field = StructField("c", IntegerType, nullable = false) - val column = Column.fromStructField(field) - assert(column.name() == "c") - assert(column.dataType() == IntegerType) - assert(!column.nullable()) - assert(column.comment() == null) - assert(column.metadataInJSON() == null) - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index 91dbebfc3ed87..ce4f5e89be2b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -315,30 +315,6 @@ class DataTypeSuite extends SparkFunSuite { checkDataTypeFromJson(structType) checkDataTypeFromDDL(structType) - test("StructField json round-trip preserves name, type, nullable, comment, metadata") { - val baseMetadata = new MetadataBuilder() - .putString("metric_view.type", "dimension") - .putString("metric_view.expr", "region") - .build() - val field = StructField("region", StringType, nullable = true, baseMetadata) - .withComment("dim col") - val parsed = StructField.fromJson(field.json) - assert(parsed.name == "region") - assert(parsed.dataType == StringType) - assert(parsed.nullable) - assert(parsed.getComment().contains("dim col")) - // `withComment` adds a "comment" key to metadata; it must round-trip alongside the - // metric_view.* keys. - assert(parsed.metadata.getString("metric_view.type") == "dimension") - assert(parsed.metadata.getString("metric_view.expr") == "region") - } - - test("StructField json round-trip with empty metadata and no comment") { - val field = StructField("c", IntegerType, nullable = false) - val parsed = StructField.fromJson(field.json) - assert(parsed == field) - } - test("fromJson throws an exception when given type string is invalid") { checkError( exception = intercept[SparkIllegalArgumentException] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index cd8ac53a58443..60bef37f4d3e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -1089,11 +1089,11 @@ object DDLUtils extends Logging { isView: Boolean): Unit = { if (!catalog.isTempView(tableMetadata.identifier)) { tableMetadata.tableType match { - case CatalogTableType.VIEW if !isView => + case t if (t == CatalogTableType.VIEW || t == CatalogTableType.METRIC_VIEW) && !isView => throw QueryCompilationErrors.cannotAlterViewWithAlterTableError( viewName = tableMetadata.identifier.table ) - case o if o != CatalogTableType.VIEW && isView => + case o if o != CatalogTableType.VIEW && o != CatalogTableType.METRIC_VIEW && isView => throw QueryCompilationErrors.cannotAlterTableWithAlterViewError( tableName = tableMetadata.identifier.table ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala index ed3a3a2a223c0..2f6b226f4e959 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala @@ -68,7 +68,8 @@ case class CreateMetricViewCommand( resolved: ResolvedIdentifier): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val name = resolved.identifier.asTableIdentifier - val analyzed = MetricViewHelper.analyzeMetricViewText(sparkSession, name, originalText) + val analyzed = MetricViewHelper.analyzeMetricViewText( + sparkSession, name.nameParts, originalText) validateUserColumns(name, analyzed) catalog.createTable( ViewHelper.prepareTable( @@ -122,15 +123,41 @@ object MetricViewHelper { tables.distinct.toSeq } + /** + * Analyzes a metric-view YAML body so the create / alter path can capture the source plan + * and its dependencies. + * + * `nameParts` is the multi-part target identifier (catalog + namespace + table). The synthetic + * [[CatalogTable]] used as analysis context still carries a [[TableIdentifier]] (capped at + * 3 parts: catalog + database + table); for multi-level v2 namespaces we collapse the + * intermediate namespace components into the synthetic `database` slot. The synthetic identifier + * is not used to resolve the view body itself, so this collapse is observationally invisible to + * the analyzed plan; `verifyTemporaryObjectsNotExists` continues to receive the full + * `nameParts` so error messages still render the multi-part form. + */ def analyzeMetricViewText( session: SparkSession, - name: TableIdentifier, + nameParts: Seq[String], viewText: String): LogicalPlan = { val analyzer = session.sessionState.analyzer + val syntheticIdent = nameParts match { + case Seq(table) => + TableIdentifier(table) + case Seq(db, table) => + TableIdentifier(table, Some(db)) + case parts => + // 3+ parts: catalog is the head, table is the last, the middle (1..n-1) collapses + // into the synthetic `database` slot. We dot-join the intermediate components so a + // human inspecting the synthetic identifier can still see them. + TableIdentifier( + parts.last, + Some(parts.slice(1, parts.length - 1).mkString(".")), + Some(parts.head)) + } // this metadata is used for analysis check, it'll be replaced during create/update with // more accurate information val tableMeta = CatalogTable( - identifier = name, + identifier = syntheticIdent, tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType(), @@ -140,7 +167,7 @@ object MetricViewHelper { tableMeta, viewText, session.sessionState.sqlParser) val analyzed = analyzer.executeAndCheck(metricViewNode, new QueryPlanningTracker) ViewHelper.verifyTemporaryObjectsNotExists( - isTemporary = false, name.nameParts, analyzed, Seq.empty) + isTemporary = false, nameParts, analyzed, Seq.empty) analyzed } } 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 58065c1793ec4..303b8820e00df 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 @@ -34,7 +34,7 @@ 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.{quoteIfNeeded, toPrettySQL, GeneratedColumn, IdentityColumn, ResolveDefaultColumns, ResolveTableConstraints, V2ExpressionBuilder} import org.apache.spark.sql.classic.SparkSession -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Dependency, DependencyList, Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, TableCapability, TableCatalog, TruncatableTable, V1Table, V1ViewInfo, ViewCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Dependency, DependencyList, Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, TableCapability, TableCatalog, TableSummary, 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} @@ -338,9 +338,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // Parse + analyze the YAML body here (during planning). This mirrors the v1 path's // late analysis in `CreateMetricViewCommand.run` -- the metric-view source plan is not // a SQL string, so it can't ride along as a regular `query` `LogicalPlan` field on the - // logical command the way `CreateView` does. + // logical command the way `CreateView` does. Pass the full multi-part name so v2 metric + // views with multi-level-namespace targets analyze correctly (`asTableIdentifier` would + // throw `requiresSinglePartNamespaceError` for namespace arity > 1). + val nameParts = (catalog.name() +: ident.namespace().toIndexedSeq) :+ ident.name() val analyzed = MetricViewHelper.analyzeMetricViewText( - session, ident.asTableIdentifier, originalText) + session, nameParts, originalText) val metricView = MetricViewFactory.fromYAML(originalText) val mergedProps = properties ++ metricView.getProperties val depParts = MetricViewHelper.collectTableDependencies(analyzed) @@ -382,6 +385,16 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat RenameV2ViewExec( catalog.asInstanceOf[ViewCatalog], ident, newName.asIdentifier) :: Nil + case ShowCreateTable(rpv @ ResolvedPersistentView(catalog, ident, _), _, _) + if rpv.info.properties.get(TableCatalog.PROP_TABLE_TYPE) == + TableSummary.METRIC_VIEW_TABLE_TYPE => + // SHOW CREATE TABLE on a metric view is explicitly unsupported: `ShowCreateV2ViewExec` + // would emit a plain `CREATE VIEW AS `, which is not a round-trippable + // metric-view DDL form (the right form is `CREATE VIEW WITH METRICS LANGUAGE + // YAML AS $$ $$`). Reject up front rather than emit lossy DDL. + throw QueryCompilationErrors.unsupportedTableOperationError( + catalog, ident, "SHOW CREATE TABLE") + case ShowCreateTable(rpv @ ResolvedPersistentView(catalog, ident, _), _, output) => val quoted = (catalog.name() +: ident.asMultipartIdentifier).map(quoteIfNeeded).mkString(".") ShowCreateV2ViewExec(output, quoted, rpv.info) :: 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 93227a01839b7..d1ceeba833ea7 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,7 +24,7 @@ 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.{CatalogV2Util, Identifier, TableCatalog, ViewInfo} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog, TableSummary, ViewInfo} import org.apache.spark.sql.errors.QueryCompilationErrors /** @@ -169,6 +169,15 @@ case class DescribeV2ViewExec( result += toCatalystRow("", "", "") result += toCatalystRow("# Detailed View Information", "", "") addIdentifierRows(result, catalogName, identifier, entityLabel = "View") + // Surface the view sub-kind so users see whether they're looking at a plain VIEW + // or a sub-kind like METRIC_VIEW. `ViewInfo`'s constructor unconditionally stamps + // `PROP_TABLE_TYPE` (defaulting to `VIEW`), so this row is always present and + // matches v1 `CatalogTable.toJsonLinkedHashMap`'s `Type` row for parity. + result += toCatalystRow( + "Type", + Option(viewInfo.properties.get(TableCatalog.PROP_TABLE_TYPE)) + .getOrElse(TableSummary.VIEW_TABLE_TYPE), + "") // 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`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index d869040eedc50..03b56cef8ec18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -378,11 +378,14 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { |$yaml |$$$$""".stripMargin) } - // CreateV2ViewExec / CreateV2MetricViewExec route this through - // `unsupportedCreateOrReplaceViewOnTableError` which maps to - // `EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE`. - assert(ex.getCondition === "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE", - s"Expected EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE, got ${ex.getCondition}: ${ex.getMessage}") + // SPARK-56655 added an analyzer-time pre-check for "ident already occupied by a table" + // before the v2 view-create exec runs, so the more specific + // `EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE` decoded by `CreateV2MetricViewExec.run`'s catch + // block is no longer reachable when a *plain* table sits at the ident -- the analyzer + // raises `TABLE_OR_VIEW_ALREADY_EXISTS` first. Both errors carry the same actionable + // signal ("can't create a view here because something else already lives at this ident"). + assert(ex.getCondition === "TABLE_OR_VIEW_ALREADY_EXISTS", + s"Expected TABLE_OR_VIEW_ALREADY_EXISTS, got ${ex.getCondition}: ${ex.getMessage}") } } @@ -433,6 +436,49 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { assert(ex.getMessage.contains("VIEWS")) } + test("CREATE VIEW ... WITH METRICS at a multi-level-namespace v2 target succeeds") { + val deepNamespace = Array("ns_a", "ns_b") + val deepMetricViewName = "mv_deep" + val fullDeepName = + s"$testCatalogName.${deepNamespace.mkString(".")}.$deepMetricViewName" + withTestCatalogTables { + // Pre-create the multi-level namespace + a source table inside it. The metric view + // *target* lives in the same multi-level namespace -- that's what exercises the + // `MetricViewHelper.analyzeMetricViewText` lift to multi-part nameParts. The pre-lift + // code path failed at `ident.asTableIdentifier` with `requiresSinglePartNamespaceError`. + sql(s"CREATE NAMESPACE IF NOT EXISTS $testCatalogName.${deepNamespace.head}") + sql(s"CREATE NAMESPACE IF NOT EXISTS " + + s"$testCatalogName.${deepNamespace.mkString(".")}") + try { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + val yaml = MetricViewFactory.toYAML(mv) + sql( + s"""CREATE VIEW $fullDeepName + |WITH METRICS + |LANGUAGE YAML + |AS + |$$$$ + |$yaml + |$$$$""".stripMargin) + + val deepIdent = Identifier.of(deepNamespace, deepMetricViewName) + val info = MetricViewRecordingCatalog.capturedViews.get(deepIdent) + assert(info != null, s"Expected ViewInfo for $deepIdent to be captured") + assert(info.properties().get(TableCatalog.PROP_TABLE_TYPE) + === TableSummary.METRIC_VIEW_TABLE_TYPE) + } finally { + scala.util.Try(sql(s"DROP VIEW IF EXISTS $fullDeepName")) + sql(s"DROP NAMESPACE IF EXISTS " + + s"$testCatalogName.${deepNamespace.mkString(".")} CASCADE") + sql(s"DROP NAMESPACE IF EXISTS $testCatalogName.${deepNamespace.head} CASCADE") + } + } + } + // ============================================================ // Section 2: Dependency extraction // ============================================================ @@ -697,18 +743,23 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // DESCRIBE TABLE EXTENDED resolves the ident through `Analyzer.lookupTableOrView`, // which calls `TableViewCatalog.loadTableOrView` once and gets back a - // `MetadataTable(ViewInfo)`. `V1Table.toCatalogTable(ViewInfo)` reconstructs the - // V1 representation; the resulting `CatalogTable.toJsonLinkedHashMap` (interface.scala) - // then emits view-context rows because `tableType == METRIC_VIEW` was added to the - // VIEW gate. Without that gate fix, "View Text" / "View Original Text" disappear. + // `MetadataTable(ViewInfo)`. The analyzer wraps it as a `ResolvedPersistentView` and + // `DataSourceV2Strategy` routes through SPARK-56655's `DescribeV2ViewExec`, which + // reads the typed `ViewInfo` directly and emits the standard "Type" / "View Text" / + // "View Current Catalog" / "View Schema Mode" / etc. rows. The "Type" row was added + // alongside this metric-view PR so `DescribeV2ViewExec` matches v1 parity for the + // v1 `CatalogTable.toJsonLinkedHashMap` `Type` row, and so users see whether they're + // looking at a plain VIEW or a sub-kind like METRIC_VIEW. val rows = sql(s"DESCRIBE TABLE EXTENDED $fullMetricViewName").collect() val rowMap = rows.map(r => r.getString(0) -> r.getString(1)).toMap assert(rowMap.contains("View Text"), s"Expected 'View Text' row in DESCRIBE EXTENDED output, got keys: ${rowMap.keys}") - assert(rowMap("View Text") === yaml, + // `DescribeV2ViewExec` writes `viewInfo.queryText` directly, so trim handles the + // leading/trailing newline the SQL `$$ ... $$` fixture inserts vs. the bare yaml body. + assert(rowMap("View Text").trim === yaml.trim, s"View Text should round-trip the YAML body, got: ${rowMap("View Text")}") - assert(rowMap.get("Type").exists(_.contains("METRIC_VIEW")), + assert(rowMap.get("Type").contains(TableSummary.METRIC_VIEW_TABLE_TYPE), s"Type row should reflect METRIC_VIEW, got: ${rowMap.get("Type")}") } } @@ -750,7 +801,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } - test("DROP TABLE on a v2 metric view throws EXPECT_TABLE_NOT_VIEW") { + test("DROP TABLE on a v2 metric view throws WRONG_COMMAND_FOR_OBJECT_TYPE") { withTestCatalogTables { val mv = MetricView( "0.1", @@ -759,15 +810,16 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { select = metricViewColumns) createMetricView(fullMetricViewName, mv) - // `DropTableExec` first probes `tableExists` (false for a view per the TableViewCatalog - // passive-filtering contract), then falls back to `viewExists` and -- when the entity - // exists as a view but a table was requested -- throws `EXPECT_TABLE_NOT_VIEW` to - // distinguish "wrong type" from "missing". + // SPARK-56655's `DropTableExec` actively rejects with `WRONG_COMMAND_FOR_OBJECT_TYPE` + // ("Use DROP VIEW instead") when a view sits at the ident, replacing the prior + // `EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE` decoding. Same actionable signal for users. val ex = intercept[AnalysisException] { sql(s"DROP TABLE $fullMetricViewName") } - assert(ex.getCondition === "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", - s"Expected EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE, got ${ex.getCondition}: ${ex.getMessage}") + assert(ex.getCondition === "WRONG_COMMAND_FOR_OBJECT_TYPE", + s"Expected WRONG_COMMAND_FOR_OBJECT_TYPE, got ${ex.getCondition}: ${ex.getMessage}") + assert(ex.getMessage.contains("DROP VIEW"), + s"Error message should mention 'DROP VIEW', got: ${ex.getMessage}") // The metric view is still present after the failed DROP TABLE. val ident = Identifier.of(Array(testNamespace), metricViewName) @@ -776,7 +828,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } - test("DROP TABLE IF EXISTS on a v2 metric view also throws EXPECT_TABLE_NOT_VIEW") { + test("DROP TABLE IF EXISTS on a v2 metric view also throws WRONG_COMMAND_FOR_OBJECT_TYPE") { withTestCatalogTables { val mv = MetricView( "0.1", @@ -786,13 +838,13 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, mv) // IF EXISTS does not silence the wrong-type error: the entity exists, just not as a - // table. (This mirrors the v1 `DropTableCommand` behavior, where `IF EXISTS` only - // short-circuits the not-found branch.) + // table. (Mirrors the v1 `DropTableCommand` behavior; `IF EXISTS` only short-circuits + // the not-found branch.) val ex = intercept[AnalysisException] { sql(s"DROP TABLE IF EXISTS $fullMetricViewName") } - assert(ex.getCondition === "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", - s"Expected EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE, got ${ex.getCondition}: ${ex.getMessage}") + assert(ex.getCondition === "WRONG_COMMAND_FOR_OBJECT_TYPE", + s"Expected WRONG_COMMAND_FOR_OBJECT_TYPE, got ${ex.getCondition}: ${ex.getMessage}") } } @@ -826,7 +878,7 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } - test("SHOW TABLES does not list v2 metric views") { + test("SHOW TABLES on a v2 TableViewCatalog lists both tables and metric views") { withTestCatalogTables { val mv = MetricView( "0.1", @@ -836,13 +888,13 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, mv) val tables = sql(s"SHOW TABLES IN $testCatalogName.$testNamespace") .collect().map(_.getString(1)).toSet - // The fixture's `events` source table is a regular table, so SHOW TABLES sees it. + // SPARK-56655 routes SHOW TABLES on a `TableViewCatalog` through `listRelationSummaries` + // so views appear alongside tables in the output (matching v1 SHOW TABLES on a session + // catalog). Pure `TableCatalog` catalogs continue to return tables only. assert(tables.contains(sourceTableName), s"SHOW TABLES should list the source table, got: $tables") - // Per the TableViewCatalog contract, SHOW TABLES returns tables only -- metric views - // belong on SHOW VIEWS instead. - assert(!tables.contains(metricViewName), - s"SHOW TABLES should not list metric views, got: $tables") + assert(tables.contains(metricViewName), + s"SHOW TABLES on a TableViewCatalog should also list metric views, got: $tables") } } From 437b71d79e45fd8caaed91f70a41515210af0e06 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Mon, 4 May 2026 13:52:05 +0000 Subject: [PATCH 10/17] [SPARK-XXXXX][SQL][FOLLOWUP] Address cloud-fan round 3 review on PR #55487 - Revert the unrelated `tableCatalog` extraction in the `DropTable` strategy case (round 2 leftover). - Single YAML parse per CREATE: `MetricViewHelper.analyzeMetricViewText` now returns `(LogicalPlan, MetricView)`, grabbing the parsed `MetricView` off the un-analyzed `MetricViewPlaceholder` so callers don't re-parse. - v1/v2 parity for `metric_view.*` descriptor properties: lift the `metricView.getProperties` merge into the v1 `createMetricViewInSessionCatalog` path so v1 `DESCRIBE TABLE EXTENDED` surfaces the same descriptor rows as v2. - v1/v2 parity for `SHOW CREATE TABLE` error class: switch the v2 path to the dedicated `UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW` error class. - Normalize v1 source dependencies to a stable 3-part `[spark_catalog, db, table]` shape via a new `qualifyV1` helper, applied in the `View` / `HiveTableRelation` / `LogicalRelation` arms of `collectTableDependencies`. Updates `TableDependency` Javadoc and the V1 source dep test accordingly. - Switch `Dependency` / `TableDependency` / `FunctionDependency` / `DependencyList` from `String[]` / `Dependency[]` to immutable `List<>` so records inherit value `equals` / `hashCode` for free (closes the reference-equality bug on array fields). Internal copies use `List.copyOf`; static factories still take varargs. - Rephrase 2 test comments in `MetricViewV2CatalogSuite` to describe the pinned invariant rather than the in-PR fix (so they age well post-merge). Signed-off-by: chen.wang --- .../sql/connector/catalog/Dependency.java | 6 +- .../sql/connector/catalog/DependencyList.java | 23 +++--- .../connector/catalog/FunctionDependency.java | 18 ++--- .../connector/catalog/TableDependency.java | 34 ++++----- .../command/metricViewCommands.scala | 58 +++++++++++---- .../datasources/v2/DataSourceV2Strategy.scala | 17 +++-- .../execution/MetricViewV2CatalogSuite.scala | 73 ++++++++++--------- 7 files changed, 130 insertions(+), 99 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java index 5a0ca939f98d8..eff717f57e3d3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java @@ -17,6 +17,8 @@ package org.apache.spark.sql.connector.catalog; +import java.util.List; + import org.apache.spark.annotation.Evolving; /** @@ -37,7 +39,7 @@ public sealed interface Dependency permits TableDependency, FunctionDependency { * components followed by the table name. */ static TableDependency table(String... nameParts) { - return new TableDependency(nameParts); + return new TableDependency(List.of(nameParts)); } /** @@ -47,6 +49,6 @@ static TableDependency table(String... nameParts) { * elements are namespace components followed by the function name. */ static FunctionDependency function(String... nameParts) { - return new FunctionDependency(nameParts); + return new FunctionDependency(List.of(nameParts)); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java index 1f64a67dc8cd9..e0d78c8b3daf2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.connector.catalog; +import java.util.List; import java.util.Objects; import org.apache.spark.annotation.Evolving; @@ -26,28 +27,26 @@ *

*

+ *

+ * {@code dependencies} is held as an immutable {@link List} so the record's auto-generated + * {@code equals}/{@code hashCode} delegate to per-element value semantics on the contained + * {@link Dependency} entries. * - * @param dependencies array of dependencies + * @param dependencies list of dependencies (immutable copy made) * @since 4.2.0 */ @Evolving -public record DependencyList(Dependency[] dependencies) { +public record DependencyList(List dependencies) { public DependencyList { Objects.requireNonNull(dependencies, "dependencies must not be null"); - dependencies = dependencies.clone(); - } - - /** Returns a defensive copy of the underlying dependencies array. */ - @Override - public Dependency[] dependencies() { - return dependencies.clone(); + dependencies = List.copyOf(dependencies); } public static DependencyList of(Dependency... dependencies) { - return new DependencyList(dependencies); + return new DependencyList(List.of(dependencies)); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java index 7ad2522172b56..f97f849b66353 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.connector.catalog; +import java.util.List; import java.util.Objects; import org.apache.spark.annotation.Evolving; @@ -26,23 +27,20 @@ *

* The dependent function is identified by its structural multi-part name. See * {@link TableDependency} for the parts-form contract. + *

+ * {@code nameParts} is held as an immutable {@link List} so the record's auto-generated + * {@code equals}/{@code hashCode} delegate to per-element value semantics. * - * @param nameParts structural multi-part identifier (defensive copy made; never empty) + * @param nameParts structural multi-part identifier (immutable copy made; never empty) * @since 4.2.0 */ @Evolving -public record FunctionDependency(String[] nameParts) implements Dependency { +public record FunctionDependency(List nameParts) implements Dependency { public FunctionDependency { Objects.requireNonNull(nameParts, "nameParts must not be null"); - if (nameParts.length == 0) { + if (nameParts.isEmpty()) { throw new IllegalArgumentException("nameParts must not be empty"); } - nameParts = nameParts.clone(); - } - - /** Returns a defensive copy of the underlying parts array. */ - @Override - public String[] nameParts() { - return nameParts.clone(); + nameParts = List.copyOf(nameParts); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java index 32a7c462b4b98..ea20c18051af0 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.connector.catalog; +import java.util.List; import java.util.Objects; import org.apache.spark.annotation.Evolving; @@ -26,31 +27,28 @@ *

* The dependent table is identified by its structural multi-part name. {@code nameParts} * arity matches the catalog's namespace depth plus one for the table name -- for a catalog - * with single-level namespaces the parts are typically - * {@code [catalog, schema, table]}; for a catalog with multi-level namespaces (e.g. Iceberg - * with {@code db1.db2}) the parts are {@code [catalog, db1, db2, ..., table]}; for sources - * referenced through a session catalog without an explicit catalog component the parts can - * be {@code [db, table]} or just {@code [table]}. The structural form preserves arity and - * is unambiguous against quoted identifiers containing a literal {@code .}; consumers that - * need a flat string should join the parts themselves with a quoting scheme appropriate to - * their wire format. + * with single-level namespaces the parts are {@code [catalog, schema, table]}; for a catalog + * with multi-level namespaces (e.g. Iceberg with {@code db1.db2}) the parts are + * {@code [catalog, db1, db2, ..., table]}; for v1 sources resolved through the session + * catalog producers should normalize to {@code [spark_catalog, db, table]} so consumers see + * a stable arity per source kind. The structural form preserves arity and is unambiguous + * against quoted identifiers containing a literal {@code .}; consumers that need a flat + * string should join the parts themselves with a quoting scheme appropriate to their wire + * format. + *

+ * {@code nameParts} is held as an immutable {@link List} so the record's auto-generated + * {@code equals}/{@code hashCode} delegate to per-element value semantics. * - * @param nameParts structural multi-part identifier (defensive copy made; never empty) + * @param nameParts structural multi-part identifier (immutable copy made; never empty) * @since 4.2.0 */ @Evolving -public record TableDependency(String[] nameParts) implements Dependency { +public record TableDependency(List nameParts) implements Dependency { public TableDependency { Objects.requireNonNull(nameParts, "nameParts must not be null"); - if (nameParts.length == 0) { + if (nameParts.isEmpty()) { throw new IllegalArgumentException("nameParts must not be empty"); } - nameParts = nameParts.clone(); - } - - /** Returns a defensive copy of the underlying parts array. */ - @Override - public String[] nameParts() { - return nameParts.clone(); + nameParts = List.copyOf(nameParts); } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala index 2f6b226f4e959..f0d627d468ddb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala @@ -21,13 +21,15 @@ import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{QueryPlanningTracker, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, SchemaUnsupported} -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HiveTableRelation} +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HiveTableRelation, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, View} +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.CatalogV2Util import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.metricview.serde.MetricView import org.apache.spark.sql.metricview.util.MetricViewPlanner import org.apache.spark.sql.types.StructType @@ -68,13 +70,17 @@ case class CreateMetricViewCommand( resolved: ResolvedIdentifier): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val name = resolved.identifier.asTableIdentifier - val analyzed = MetricViewHelper.analyzeMetricViewText( + val (analyzed, metricView) = MetricViewHelper.analyzeMetricViewText( sparkSession, name.nameParts, originalText) validateUserColumns(name, analyzed) + // Merge the descriptor `metric_view.*` properties (`from.type`, `from.name`/`from.sql`, + // `where`) into the user-supplied properties so v1 DESCRIBE TABLE EXTENDED surfaces the + // same descriptor rows as the v2 path in `DataSourceV2Strategy`. + val mergedProps = properties ++ metricView.getProperties catalog.createTable( ViewHelper.prepareTable( sparkSession, name, Some(originalText), analyzed, userSpecifiedColumns, - properties, SchemaUnsupported, comment, + mergedProps, SchemaUnsupported, comment, None, isMetricView = true), ignoreIfExists = allowExisting) Seq.empty @@ -91,9 +97,13 @@ object MetricViewHelper { /** * Walks the analyzed plan to collect direct table/view dependencies. Each dependency is - * returned as a structural multi-part name (`Seq[String]`); arity is preserved per source - * so consumers can reason about catalog / namespace / table boundaries without parsing a - * dot-flattened string. + * returned as a structural multi-part name (`Seq[String]`); v1 sources (resolved through + * the session catalog) are normalized to a stable 3-part shape + * `[spark_catalog, db, table]` -- `TableIdentifier.nameParts` returns 1, 2, or 3 parts + * depending on whether the analyzer captured the catalog / database, so without + * normalization the same source can produce a different shape across runs. v2 sources + * already arrive fully qualified (catalog + namespace + table) and are returned as-is so + * multi-level namespaces survive. * * Stops recursion at relation leaf nodes and persistent `View` nodes so only direct * (not transitive) dependencies are recorded. @@ -102,16 +112,16 @@ object MetricViewHelper { val tables = scala.collection.mutable.ArrayBuffer.empty[Seq[String]] def traverse(p: LogicalPlan): Unit = p match { case v: View if !v.isTempView => - tables += v.desc.identifier.nameParts + tables += qualifyV1(v.desc.identifier.nameParts) case r: DataSourceV2Relation if r.catalog.isDefined && r.identifier.isDefined => val ident = r.identifier.get // V2 catalogs may have multi-level namespaces; preserve the full arity rather than // dot-joining the namespace into a single component. tables += (r.catalog.get.name() +: ident.namespace().toIndexedSeq) :+ ident.name() case r: HiveTableRelation => - tables += r.tableMeta.identifier.nameParts + tables += qualifyV1(r.tableMeta.identifier.nameParts) case r: LogicalRelation if r.catalogTable.isDefined => - tables += r.catalogTable.get.identifier.nameParts + tables += qualifyV1(r.catalogTable.get.identifier.nameParts) case other => other.children.foreach(traverse) other.expressions.foreach(_.foreach { @@ -123,9 +133,25 @@ object MetricViewHelper { tables.distinct.toSeq } + /** + * Normalizes v1 source identifiers to a stable 3-part `[spark_catalog, db, table]` shape. + * `TableIdentifier.nameParts` may return 1, 2, or 3 parts depending on whether the analyzer + * captured the catalog / database components, which would otherwise leak through to + * dependency consumers as nondeterministic arity. + */ + private def qualifyV1(parts: Seq[String]): Seq[String] = parts match { + case Seq(t) => Seq(SESSION_CATALOG_NAME, SessionCatalog.DEFAULT_DATABASE, t) + case Seq(db, t) => Seq(SESSION_CATALOG_NAME, db, t) + case Seq(_, _, _) => parts + case other => other // Unexpected arity; pass through unchanged. + } + /** * Analyzes a metric-view YAML body so the create / alter path can capture the source plan - * and its dependencies. + * and its dependencies. Returns the analyzed plan together with the parsed [[MetricView]] + * descriptor (the latter is grabbed off the un-analyzed [[MetricViewPlaceholder]] before + * the analyzer rewrites it away, so callers needing the descriptor for property emission + * don't have to re-parse the YAML). * * `nameParts` is the multi-part target identifier (catalog + namespace + table). The synthetic * [[CatalogTable]] used as analysis context still carries a [[TableIdentifier]] (capped at @@ -138,7 +164,7 @@ object MetricViewHelper { def analyzeMetricViewText( session: SparkSession, nameParts: Seq[String], - viewText: String): LogicalPlan = { + viewText: String): (LogicalPlan, MetricView) = { val analyzer = session.sessionState.analyzer val syntheticIdent = nameParts match { case Seq(table) => @@ -163,11 +189,15 @@ object MetricViewHelper { schema = new StructType(), viewOriginalText = Some(viewText), viewText = Some(viewText)) - val metricViewNode = MetricViewPlanner.planWrite( + val placeholder = MetricViewPlanner.planWrite( tableMeta, viewText, session.sessionState.sqlParser) - val analyzed = analyzer.executeAndCheck(metricViewNode, new QueryPlanningTracker) + // Grab the parsed descriptor BEFORE analysis -- the placeholder gets replaced by + // ResolvedMetricView during analyzer rules, after which `MetricView` is no longer + // recoverable from the plan tree. + val metricView = placeholder.desc + val analyzed = analyzer.executeAndCheck(placeholder, new QueryPlanningTracker) ViewHelper.verifyTemporaryObjectsNotExists( isTemporary = false, nameParts, analyzed, Seq.empty) - analyzed + (analyzed, metricView) } } 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 303b8820e00df..ed86ef96a4434 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 @@ -49,7 +49,6 @@ import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRe import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH -import org.apache.spark.sql.metricview.serde.MetricViewFactory import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ArrayImplicits._ @@ -342,9 +341,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // views with multi-level-namespace targets analyze correctly (`asTableIdentifier` would // throw `requiresSinglePartNamespaceError` for namespace arity > 1). val nameParts = (catalog.name() +: ident.namespace().toIndexedSeq) :+ ident.name() - val analyzed = MetricViewHelper.analyzeMetricViewText( + val (analyzed, metricView) = MetricViewHelper.analyzeMetricViewText( session, nameParts, originalText) - val metricView = MetricViewFactory.fromYAML(originalText) val mergedProps = properties ++ metricView.getProperties val depParts = MetricViewHelper.collectTableDependencies(analyzed) val deps = if (depParts.nonEmpty) { @@ -391,9 +389,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // SHOW CREATE TABLE on a metric view is explicitly unsupported: `ShowCreateV2ViewExec` // would emit a plain `CREATE VIEW AS `, which is not a round-trippable // metric-view DDL form (the right form is `CREATE VIEW WITH METRICS LANGUAGE - // YAML AS $$ $$`). Reject up front rather than emit lossy DDL. - throw QueryCompilationErrors.unsupportedTableOperationError( - catalog, ident, "SHOW CREATE TABLE") + // YAML AS $$ $$`). Reject up front with the same dedicated error class the v1 + // path uses (`UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW`) so users see the same + // actionable message regardless of catalog kind. + val quoted = (catalog.name() +: ident.asMultipartIdentifier) + .map(quoteIfNeeded).mkString(".") + throw QueryCompilationErrors.showCreateTableNotSupportedOnMetricViewError(quoted) case ShowCreateTable(rpv @ ResolvedPersistentView(catalog, ident, _), _, output) => val quoted = (catalog.name() +: ident.asMultipartIdentifier).map(quoteIfNeeded).mkString(".") @@ -604,9 +605,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } case DropTable(r: ResolvedIdentifier, ifExists, purge) => - val tableCatalog = r.catalog.asTableCatalog val invalidateFunc = () => CommandUtils.uncacheTableOrView(session, r) - DropTableExec(tableCatalog, r.identifier, ifExists, purge, invalidateFunc) :: Nil + DropTableExec( + r.catalog.asTableCatalog, r.identifier, ifExists, purge, invalidateFunc) :: Nil case _: NoopCommand => LocalTableScanExec(Nil, Nil, None) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 03b56cef8ec18..6fead60e4983f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -153,9 +153,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val deps = info.viewDependencies() assert(deps != null) - assert(deps.dependencies().length === 1) - val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().toSeq === + assert(deps.dependencies().size() === 1) + val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().asScala.toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -209,9 +209,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { assert(props.get(TableCatalog.PROP_COMMENT) === "my mv") val deps = info.viewDependencies() - assert(deps != null && deps.dependencies().length === 1) - val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().toSeq === + assert(deps != null && deps.dependencies().size() === 1) + val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().asScala.toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -250,8 +250,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { where = None, select = metricViewColumns) val yaml = MetricViewFactory.toYAML(metricView) - // Give both columns new names, and a comment on each. Without the `retainMetadata` - // fix to `ViewHelper.aliasPlan`, the metric_view.* keys disappear here. + // Pins aliasPlan(retainMetadata = true): metric_view.* keys must survive a column + // rename with comments. sql( s"""CREATE VIEW $fullMetricViewName (reg COMMENT 'region alias', n COMMENT 'count') |WITH METRICS @@ -313,9 +313,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { assert(finalInfo.queryText().trim === replacementYaml.trim) assert(finalInfo.properties().get(MetricView.PROP_WHERE) === "count > 100") val deps = finalInfo.viewDependencies() - assert(deps != null && deps.dependencies().length === 1) - val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().toSeq === + assert(deps != null && deps.dependencies().size() === 1) + val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().asScala.toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -505,8 +505,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val deps = capturedViewInfo().viewDependencies() assert(deps != null) - val depParts = - deps.dependencies().map(_.asInstanceOf[TableDependency].nameParts().toSeq).toSet + val depParts = deps.dependencies().asScala + .map(_.asInstanceOf[TableDependency].nameParts().asScala.toSeq).toSet assert(depParts === Set( Seq(testCatalogName, testNamespace, sourceTableName), Seq(testCatalogName, testNamespace, "customers")), @@ -530,11 +530,11 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, metricView) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().length === 1, + assert(deps != null && deps.dependencies().size() === 1, s"Expected 1 deduplicated dependency, got " + - s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") - val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().toSeq === + s"${Option(deps).map(_.dependencies().size()).getOrElse(0)}") + val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().asScala.toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -555,11 +555,11 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, metricView) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().length === 1, + assert(deps != null && deps.dependencies().size() === 1, s"Expected 1 deduplicated dependency for self-join, got " + - s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") - val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().toSeq === + s"${Option(deps).map(_.dependencies().size()).getOrElse(0)}") + val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().asScala.toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -580,15 +580,18 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, mv) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().length === 1) - val parts = deps.dependencies()(0).asInstanceOf[TableDependency].nameParts().toSeq - // For a session-catalog source, `TableIdentifier.nameParts` includes catalog + db + - // table when the catalog is set; here we expect at least 2 parts (`db.table`) and - // up to 3 (`spark_catalog.db.table`) -- both are valid producer outputs depending - // on whether the analyzer captured the session-catalog component. + assert(deps != null && deps.dependencies().size() === 1) + val parts = + deps.dependencies().get(0).asInstanceOf[TableDependency].nameParts().asScala.toSeq + // `MetricViewHelper.qualifyV1` normalizes any `TableIdentifier.nameParts` shape + // (1, 2, or 3 parts depending on what the analyzer captured) to the stable + // `[spark_catalog, db, table]` shape so downstream consumers see deterministic + // arity per source kind. + assert(parts.length === 3, + s"V1 nameParts should normalize to exactly 3 parts, got ${parts.length}: $parts") + assert(parts.head === "spark_catalog", + s"V1 nameParts head should be the session-catalog name, got $parts") assert(parts.last === v1Source, s"Last part should be the table name, got $parts") - assert(parts.length >= 2 && parts.length <= 3, - s"V1 nameParts arity should be 2 or 3, got ${parts.length}: $parts") } } finally { sql(s"DROP TABLE IF EXISTS $v1Source") @@ -614,8 +617,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, mv) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().length === 1) - val parts = deps.dependencies()(0).asInstanceOf[TableDependency].nameParts().toSeq + assert(deps != null && deps.dependencies().size() === 1) + val parts = + deps.dependencies().get(0).asInstanceOf[TableDependency].nameParts().asScala.toSeq assert(parts === Seq(testCatalogName, multiNamespace(0), multiNamespace(1), multiTable), s"Multi-level nameParts should preserve every namespace component, got $parts") } finally { @@ -746,10 +750,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // `MetadataTable(ViewInfo)`. The analyzer wraps it as a `ResolvedPersistentView` and // `DataSourceV2Strategy` routes through SPARK-56655's `DescribeV2ViewExec`, which // reads the typed `ViewInfo` directly and emits the standard "Type" / "View Text" / - // "View Current Catalog" / "View Schema Mode" / etc. rows. The "Type" row was added - // alongside this metric-view PR so `DescribeV2ViewExec` matches v1 parity for the - // v1 `CatalogTable.toJsonLinkedHashMap` `Type` row, and so users see whether they're - // looking at a plain VIEW or a sub-kind like METRIC_VIEW. + // "View Current Catalog" / "View Schema Mode" / etc. rows. Pins `DescribeV2ViewExec` + // emits a "Type" row matching v1 `CatalogTable.toJsonLinkedHashMap` parity, so users + // can distinguish a plain VIEW from a sub-kind like METRIC_VIEW. val rows = sql(s"DESCRIBE TABLE EXTENDED $fullMetricViewName").collect() val rowMap = rows.map(r => r.getString(0) -> r.getString(1)).toMap From 66b13ebc5d6c821425d9dde9abf1c51efa915ecf Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Mon, 4 May 2026 21:47:46 +0000 Subject: [PATCH 11/17] [SPARK-XXXXX][SQL][FOLLOWUP] Fix CI failures from round 3 follow-up Two CI fixes for the previous round-3 review commit: - Scalastyle: replace the multi-space alignment before `=>` in `MetricViewHelper.qualifyV1`'s pattern-match arms with a single space (Scalastyle's `NoWhitespaceBeforeLeftBracketChecker` flags lined-up `=>` arrows). - Test: update `MetricViewV2CatalogSuite`'s SHOW CREATE TABLE assertion to expect `UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW` (the dedicated error class introduced in the round-3 commit) instead of the previously-used generic `UNSUPPORTED_FEATURE.TABLE_OPERATION`. Other CI job failures on the prior run are unrelated infrastructure flakes (Kerberos PREAUTH on Docker integration tests, minikube termination on Kubernetes IT, RemoteClassLoaderError on Connect's `SparkSessionE2ESuite`). Signed-off-by: chen.wang --- .../execution/command/metricViewCommands.scala | 8 ++++---- .../execution/MetricViewV2CatalogSuite.scala | 18 ++++++++++-------- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala index f0d627d468ddb..5937ad2300cd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/metricViewCommands.scala @@ -140,10 +140,10 @@ object MetricViewHelper { * dependency consumers as nondeterministic arity. */ private def qualifyV1(parts: Seq[String]): Seq[String] = parts match { - case Seq(t) => Seq(SESSION_CATALOG_NAME, SessionCatalog.DEFAULT_DATABASE, t) - case Seq(db, t) => Seq(SESSION_CATALOG_NAME, db, t) - case Seq(_, _, _) => parts - case other => other // Unexpected arity; pass through unchanged. + case Seq(t) => Seq(SESSION_CATALOG_NAME, SessionCatalog.DEFAULT_DATABASE, t) + case Seq(db, t) => Seq(SESSION_CATALOG_NAME, db, t) + case Seq(_, _, _) => parts + case other => other // Unexpected arity; pass through unchanged. } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 6fead60e4983f..554e89b3fb562 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -860,18 +860,20 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { select = metricViewColumns) createMetricView(fullMetricViewName, mv) - // SHOW CREATE TABLE on a v2 view (including metric views) is rejected by - // DataSourceV2Strategy via `unsupportedTableOperationError(...)`. There's no - // round-trippable `CREATE VIEW ... WITH METRICS` form yet, so explicit "unsupported" - // is the right answer rather than emitting a misleading plain `CREATE VIEW ...`. + // SHOW CREATE TABLE on a metric view is rejected with the dedicated + // UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW error class (same one the v1 path uses + // in `tables.scala`'s `ShowCreateTableCommand`), so the message is identical no matter + // which catalog kind owns the view. There's no round-trippable + // `CREATE VIEW ... WITH METRICS` form yet, so explicit "unsupported" is the right + // answer rather than emitting a misleading plain `CREATE VIEW ...`. val ex = intercept[AnalysisException] { sql(s"SHOW CREATE TABLE $fullMetricViewName") } - assert(ex.getCondition === "UNSUPPORTED_FEATURE.TABLE_OPERATION", - s"Expected UNSUPPORTED_FEATURE.TABLE_OPERATION, got " + + assert(ex.getCondition === "UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW", + s"Expected UNSUPPORTED_SHOW_CREATE_TABLE.ON_METRIC_VIEW, got " + s"${ex.getCondition}: ${ex.getMessage}") - assert(ex.getMessage.contains("SHOW CREATE TABLE"), - s"Error message should mention 'SHOW CREATE TABLE', got: ${ex.getMessage}") + assert(ex.getMessage.contains("metric view"), + s"Error message should mention 'metric view', got: ${ex.getMessage}") } } From e980f2442750ce9e01dd616bed5c803a99cd6e52 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Tue, 5 May 2026 13:21:31 +0000 Subject: [PATCH 12/17] [SPARK-XXXXX][SQL][FOLLOWUP] Address cloud-fan round 4 review on PR #55487 - DataSourceV2Strategy: empty dep list -> Some(DependencyList.of()), not None, so ViewInfo.viewDependencies semantics match its Javadoc ("null = not supplied", "empty = supplied but none"). Metric-view CREATE always *computes* deps, so the right empty representation is Some(empty list). - Dependency.java: add Javadoc note that today's producers emit only TableDependency; FunctionDependency / Dependency.function(...) are groundwork for future producers (e.g. SQL UDF dep tracking). - TableDependency.java: insert missing comma after "session catalog" in the parts-form Javadoc. - MetricViewV2CatalogSuite: drop "DBR" terminology, fix the broken "Pins ..." sentence in the DESCRIBE EXTENDED test, and add an ALTER VIEW RENAME TO test that pins the existing RenameV2ViewExec wiring (catalog renameView relocates both the views entry and the captured ViewInfo entry; metric-view kind survives the rename). Signed-off-by: chen.wang --- .../sql/connector/catalog/Dependency.java | 6 +++ .../connector/catalog/TableDependency.java | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 11 +++-- .../execution/MetricViewV2CatalogSuite.scala | 44 +++++++++++++++++-- 4 files changed, 55 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java index eff717f57e3d3..a39fa946f0dc5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java @@ -26,6 +26,12 @@ *

* A dependency is one of: {@link TableDependency} or {@link FunctionDependency}. The * {@code sealed} declaration enforces this structurally. + *

+ * Note: today the only producer in Spark itself is metric-view dependency extraction, which + * emits {@link TableDependency} only. {@link FunctionDependency} and the + * {@link #function(String...)} factory are exposed as groundwork for future producers + * (e.g. SQL UDF dependency tracking); consumers iterating a {@link DependencyList} received + * from Spark today should expect to see only {@link TableDependency} instances. * * @since 4.2.0 */ diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java index ea20c18051af0..717a6d9f61091 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java @@ -30,7 +30,7 @@ * with single-level namespaces the parts are {@code [catalog, schema, table]}; for a catalog * with multi-level namespaces (e.g. Iceberg with {@code db1.db2}) the parts are * {@code [catalog, db1, db2, ..., table]}; for v1 sources resolved through the session - * catalog producers should normalize to {@code [spark_catalog, db, table]} so consumers see + * catalog, producers should normalize to {@code [spark_catalog, db, table]} so consumers see * a stable arity per source kind. The structural form preserves arity and is unambiguous * against quoted identifiers containing a literal {@code .}; consumers that need a flat * string should join the parts themselves with a quoting scheme appropriate to their wire 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 ed86ef96a4434..0e826ae56e40a 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 @@ -345,10 +345,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat session, nameParts, originalText) val mergedProps = properties ++ metricView.getProperties val depParts = MetricViewHelper.collectTableDependencies(analyzed) - val deps = if (depParts.nonEmpty) { - Some(DependencyList.of( - depParts.map(parts => Dependency.table(parts: _*)): _*)) - } else None + // Always emit a `Some(DependencyList)` for metric views (even when `depParts` is empty, + // e.g. `SQLSource("SELECT 1 AS x")`): per `ViewInfo.viewDependencies()` Javadoc, `null` + // means "no dependency list was supplied" while an empty list means "supplied but the + // object has none". Metric-view CREATE always *computes* deps, so the right empty + // representation is `Some(empty list)`, not `None`. + val deps = Some(DependencyList.of( + depParts.map(parts => Dependency.table(parts: _*)): _*)) CreateV2MetricViewExec(viewCatalog, ident, userSpecifiedColumns, comment, mergedProps, originalText, analyzed, allowExisting, replace, deps) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index 554e89b3fb562..aa8be6dab6493 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -172,7 +172,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val info = capturedViewInfo() val props = info.properties() - // metric_view.* descriptive properties (mirrors DBR SingleSourceMetricView). + // metric_view.* descriptive properties (mirrors the canonical metric-view property + // layout). assert(props.get(MetricView.PROP_FROM_TYPE) === "ASSET") assert(props.get(MetricView.PROP_FROM_NAME) === fullSourceTableName) assert(props.get(MetricView.PROP_FROM_SQL) === null) @@ -750,8 +751,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { // `MetadataTable(ViewInfo)`. The analyzer wraps it as a `ResolvedPersistentView` and // `DataSourceV2Strategy` routes through SPARK-56655's `DescribeV2ViewExec`, which // reads the typed `ViewInfo` directly and emits the standard "Type" / "View Text" / - // "View Current Catalog" / "View Schema Mode" / etc. rows. Pins `DescribeV2ViewExec` - // emits a "Type" row matching v1 `CatalogTable.toJsonLinkedHashMap` parity, so users + // "View Current Catalog" / "View Schema Mode" / etc. rows. Pins that `DescribeV2ViewExec` + // emits a "Type" row for parity with v1 `CatalogTable.toJsonLinkedHashMap`, so users // can distinguish a plain VIEW from a sub-kind like METRIC_VIEW. val rows = sql(s"DESCRIBE TABLE EXTENDED $fullMetricViewName").collect() val rowMap = rows.map(r => r.getString(0) -> r.getString(1)).toMap @@ -883,6 +884,43 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { } } + test("ALTER VIEW RENAME TO ... succeeds and preserves metric view metadata") { + withTestCatalogTables { + val mv = MetricView( + "0.1", + AssetSource(fullSourceTableName), + where = None, + select = metricViewColumns) + createMetricView(fullMetricViewName, mv) + val renamedFull = s"$testCatalogName.$testNamespace.mv_renamed" + try { + // RenameTable on a `ResolvedPersistentView` is routed by `DataSourceV2Strategy` to + // `RenameV2ViewExec`, which calls `ViewCatalog.renameView` -- the fixture + // `MetricViewRecordingCatalog.renameView` relocates both the `views` entry and the + // `capturedViews` entry under the new ident. Pin the wiring end-to-end so the + // metric view kind survives the rename. + sql(s"ALTER VIEW $fullMetricViewName RENAME TO $renamedFull") + + // Old ident is gone from the v2 catalog -- DESCRIBE should fail to resolve. + val oldEx = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $fullMetricViewName").collect() + } + assert(oldEx.getCondition === "TABLE_OR_VIEW_NOT_FOUND", + s"Expected TABLE_OR_VIEW_NOT_FOUND for the old ident, got " + + s"${oldEx.getCondition}: ${oldEx.getMessage}") + + // New ident loads through `TableViewCatalog.loadTableOrView` and surfaces the same + // metric-view kind on `DESCRIBE TABLE EXTENDED`. + val rows = sql(s"DESCRIBE TABLE EXTENDED $renamedFull").collect() + val rowMap = rows.map(r => r.getString(0) -> r.getString(1)).toMap + assert(rowMap.get("Type").contains(TableSummary.METRIC_VIEW_TABLE_TYPE), + s"Renamed view should still be a METRIC_VIEW, got Type=${rowMap.get("Type")}") + } finally { + sql(s"DROP VIEW IF EXISTS $renamedFull") + } + } + } + test("SHOW TABLES on a v2 TableViewCatalog lists both tables and metric views") { withTestCatalogTables { val mv = MetricView( From fd3d8b25850e99e8ac8be50c9b06123377f06137 Mon Sep 17 00:00:00 2001 From: Chen Wang <90653848+chenwang-databricks@users.noreply.github.com> Date: Tue, 5 May 2026 11:10:02 -0400 Subject: [PATCH 13/17] Update sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala Co-authored-by: Wenchen Fan --- .../sql/execution/datasources/v2/DataSourceV2Strategy.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 0e826ae56e40a..69a01baa9f070 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 @@ -346,8 +346,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val mergedProps = properties ++ metricView.getProperties val depParts = MetricViewHelper.collectTableDependencies(analyzed) // Always emit a `Some(DependencyList)` for metric views (even when `depParts` is empty, - // e.g. `SQLSource("SELECT 1 AS x")`): per `ViewInfo.viewDependencies()` Javadoc, `null` - // means "no dependency list was supplied" while an empty list means "supplied but the + // e.g. `SQLSource("SELECT 1 AS x")`): per `DependencyList`'s contract, `null` means + // "no dependency list was supplied" while an empty list means "supplied but the // object has none". Metric-view CREATE always *computes* deps, so the right empty // representation is `Some(empty list)`, not `None`. val deps = Some(DependencyList.of( From 90deee1949c65ccc6ccd21992115d54f5fe132f2 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Wed, 6 May 2026 02:14:16 +0000 Subject: [PATCH 14/17] [SPARK-XXXXX][SQL][FOLLOWUP] Drop varargs in Dependency API; port DBR isViewLike Address cloud-fan late catch on PR #55487 (https://github.com/apache/spark/pull/55487#discussion_r3189677004) plus port DBR's `CatalogTable.isViewLike` helper to OSS to reduce divergence. - Dependency.table / Dependency.function / DependencyList.of: switch from `String... nameParts` / `Dependency... dependencies` varargs to `List` / `List` parameters. Eliminates the `: _*` splat and transient `String[]` / `Dependency[]` array allocation on every call. Records still hold immutable List internally via `List.copyOf` in the canonical constructor, so wire / value semantics are unchanged. - DataSourceV2Strategy: update the sole producer call site to construct via `.asJava` instead of `: _*`, with an explicit `Seq[Dependency]` ascription so the Java invariance + Scala covariance gap is bridged before `.asJava`. - CatalogTable.isViewLike: new instance method (mirrors DBR signature exactly so the body is the only OSS/DBR diff: today VIEW or METRIC_VIEW; DBR also includes MATERIALIZED_VIEW and STREAMING_TABLE), plus a companion `CatalogTable.isViewLike(t: CatalogTableType)` for callers that only have a `CatalogTableType` (`SessionCatalog.isView`, `ddl.scala::verifyAlterTableType`, drop-table type check, and `HiveClientImpl.toHiveTableType`). - Replace all 17 inline `tableType == VIEW || tableType == METRIC_VIEW` disjunctions and 1 `CatalogTableType.VIEW | CatalogTableType.METRIC_VIEW` pattern-alternation across catalyst / core / hive with the new helpers. Single source of truth -- adding new view-like types in the future is a one-line change in `isViewLike`. - Drop now-unused `CatalogTableType` imports from 6 files where the only reference was the just-removed disjunction. Signed-off-by: chen.wang --- .../sql/connector/catalog/Dependency.java | 8 +++---- .../sql/connector/catalog/DependencyList.java | 4 ++-- .../sql/catalyst/analysis/Analyzer.scala | 3 +-- .../analysis/RelationResolution.scala | 4 +--- .../catalyst/catalog/InMemoryCatalog.scala | 3 +-- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +-- .../sql/catalyst/catalog/interface.scala | 21 ++++++++++++++++++- .../spark/sql/classic/DataStreamWriter.scala | 5 ++--- .../command/AnalyzeColumnCommand.scala | 5 ++--- .../command/AnalyzePartitionCommand.scala | 5 ++--- .../sql/execution/command/CommandUtils.scala | 5 ++--- .../command/DescribeRelationJsonCommand.scala | 5 ++--- .../spark/sql/execution/command/ddl.scala | 8 +++---- .../spark/sql/execution/command/tables.scala | 9 +++----- .../sql/execution/datasources/rules.scala | 3 +-- .../datasources/v2/DataSourceV2Strategy.scala | 5 +++-- .../datasources/v2/V2SessionCatalog.scala | 3 +-- .../spark/sql/hive/HiveExternalCatalog.scala | 6 ++---- .../sql/hive/client/HiveClientImpl.scala | 2 +- 19 files changed, 55 insertions(+), 52 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java index a39fa946f0dc5..6349dc1b05c48 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java @@ -44,8 +44,8 @@ public sealed interface Dependency permits TableDependency, FunctionDependency { * the first element is typically the catalog name and subsequent elements are namespace * components followed by the table name. */ - static TableDependency table(String... nameParts) { - return new TableDependency(List.of(nameParts)); + static TableDependency table(List nameParts) { + return new TableDependency(nameParts); } /** @@ -54,7 +54,7 @@ static TableDependency table(String... nameParts) { * catalog-managed functions the first element is typically the catalog name and subsequent * elements are namespace components followed by the function name. */ - static FunctionDependency function(String... nameParts) { - return new FunctionDependency(List.of(nameParts)); + static FunctionDependency function(List nameParts) { + return new FunctionDependency(nameParts); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java index e0d78c8b3daf2..a790cfee91a05 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java @@ -46,7 +46,7 @@ public record DependencyList(List dependencies) { dependencies = List.copyOf(dependencies); } - public static DependencyList of(Dependency... dependencies) { - return new DependencyList(List.of(dependencies)); + public static DependencyList of(List dependencies) { + return new DependencyList(dependencies); } } 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 17c21a9fbf8e9..df7f5c662d8f8 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 @@ -1226,8 +1226,7 @@ class Analyzer( ) { CatalogV2Util.loadTable(catalog, ident).map { case v1Table: V1Table if CatalogV2Util.isSessionCatalog(catalog) && - (v1Table.v1Table.tableType == CatalogTableType.VIEW || - v1Table.v1Table.tableType == CatalogTableType.METRIC_VIEW) => + v1Table.v1Table.isViewLike => val v1Ident = v1Table.catalogTable.identifier val v2Ident = Identifier.of(v1Ident.database.toArray, v1Ident.identifier) ResolvedPersistentView( 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 d69d9cc1492e7..ef5862547574b 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 @@ -24,7 +24,6 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.catalog.{ CatalogTable, - CatalogTableType, TemporaryViewRelation, UnresolvedCatalogRelation } @@ -398,8 +397,7 @@ class RelationResolution( timeTravelSpec: Option[TimeTravelSpec]): Option[LogicalPlan] = { def createDataSourceV1Scan(v1Table: CatalogTable): LogicalPlan = { if (isStreaming) { - if (v1Table.tableType == CatalogTableType.VIEW || - v1Table.tableType == CatalogTableType.METRIC_VIEW) { + if (v1Table.isViewLike) { throw QueryCompilationErrors.permanentViewNotSupportedByStreamingReadingAPIError( ident.quoted ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 5536455c76c63..1aa5c483db88b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -395,8 +395,7 @@ class InMemoryCatalog( override def listViews(db: String, pattern: String): Seq[String] = synchronized { requireDbExists(db) val views = catalog(db).tables.filter { case (_, t) => - t.table.tableType == CatalogTableType.VIEW || - t.table.tableType == CatalogTableType.METRIC_VIEW + t.table.isViewLike }.keySet StringUtils.filterPattern(views.toSeq.sorted, pattern) } 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 41404b784580d..32aa8cccbd93a 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 @@ -1256,8 +1256,7 @@ class SessionCatalog( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val ident = nameParts.asTableIdentifier try { - val t = getTempViewOrPermanentTableMetadata(ident).tableType - t == CatalogTableType.VIEW || t == CatalogTableType.METRIC_VIEW + getTempViewOrPermanentTableMetadata(ident).isViewLike } catch { case _: NoSuchTableException => false case _: NoSuchNamespaceException => false 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 17c2a00279932..ff390f14fcd3e 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 @@ -461,6 +461,14 @@ case class CatalogTable( */ def fullIdent: Seq[String] = multipartIdentifier.getOrElse(identifier.nameParts) + /** + * Returns whether this table behaves like a view at resolution / DDL time. Today: VIEW or + * METRIC_VIEW. Forks may extend this set (e.g. DBR also includes MATERIALIZED_VIEW and + * STREAMING_TABLE), so call sites that need a uniform "is this view-like?" check should + * prefer this helper over inline disjunctions on `tableType`. + */ + def isViewLike: Boolean = CatalogTable.isViewLike(tableType) + /** * schema of this table's partition columns */ @@ -667,7 +675,7 @@ case class CatalogTable( if (comment.isDefined) map += "Comment" -> JString(comment.get) if (collation.isDefined) map += "Collation" -> JString(collation.get) - if (tableType == CatalogTableType.VIEW || tableType == CatalogTableType.METRIC_VIEW) { + if (isViewLike) { if (viewText.isDefined) { map += "View Text" -> JString(viewText.get) } @@ -758,6 +766,17 @@ object CatalogTable { table.tableType == CatalogTableType.METRIC_VIEW } + /** + * Type-only form of [[CatalogTable.isViewLike]]; returns whether the given table type + * behaves like a view at resolution / DDL time. Use this overload when you have a + * [[CatalogTableType]] but no surrounding [[CatalogTable]] (e.g. inside `match`/`case` + * patterns or [[org.apache.spark.sql.catalyst.catalog.SessionCatalog.isView]]). Body kept + * in sync with the instance method. + */ + def isViewLike(tableType: CatalogTableType): Boolean = { + tableType == CatalogTableType.VIEW || tableType == CatalogTableType.METRIC_VIEW + } + // Convert the current catalog and namespace to properties. def catalogAndNamespaceToProps( currentCatalog: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala index fca3a31c3bd7e..3e27605940383 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.sql.{streaming, Dataset => DS, ForeachWriter} import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, OptionList, UnresolvedTableSpec} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes @@ -190,8 +190,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) extends streaming.D val tableInstance = catalog.asTableCatalog.loadTable(identifier) def writeToV1Table(table: CatalogTable): StreamingQuery = { - if (table.tableType == CatalogTableType.VIEW || - table.tableType == CatalogTableType.METRIC_VIEW) { + if (table.isViewLike) { throw QueryCompilationErrors.streamingIntoViewNotSupportedError(tableName) } require(table.provider.isDefined) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 15ebd77d07cdc..924e8b820a7c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.CatalogStatistics import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.classic.ClassicConversions.castToImpl @@ -104,8 +104,7 @@ case class AnalyzeColumnCommand( private def analyzeColumnInCatalog(sparkSession: SparkSession): Unit = { val sessionState = sparkSession.sessionState val tableMeta = sessionState.catalog.getTableMetadata(tableIdent) - if (tableMeta.tableType == CatalogTableType.VIEW || - tableMeta.tableType == CatalogTableType.METRIC_VIEW) { + if (tableMeta.isViewLike) { // Analyzes a catalog view if the view is cached val plan = sparkSession.table(tableIdent.quotedString).logicalPlan if (!analyzeColumnInCachedData(plan, sparkSession)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala index e6caf1a6b218c..39169f8d22df4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.errors.QueryCompilationErrors @@ -75,8 +75,7 @@ case class AnalyzePartitionCommand( val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) val tableMeta = sessionState.catalog.getTableMetadata(tableIdentWithDB) - if (tableMeta.tableType == CatalogTableType.VIEW || - tableMeta.tableType == CatalogTableType.METRIC_VIEW) { + if (tableMeta.isViewLike) { throw QueryCompilationErrors.analyzeTableNotSupportedOnViewsError() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index 559d6917d275c..b9ff24139f07d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -29,7 +29,7 @@ import org.apache.spark.internal.LogKeys.{COUNT, DATABASE_NAME, ERROR, TABLE_NAM import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTable, CatalogTablePartition, CatalogTableType, ExternalCatalogUtils} +import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTable, CatalogTablePartition, ExternalCatalogUtils} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -240,8 +240,7 @@ object CommandUtils extends Logging { val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) val tableMeta = sessionState.catalog.getTableMetadata(tableIdentWithDB) - if (tableMeta.tableType == CatalogTableType.VIEW || - tableMeta.tableType == CatalogTableType.METRIC_VIEW) { + if (tableMeta.isViewLike) { // Analyzes a catalog view if the view is cached val table = sparkSession.table(tableIdent.quotedString) val cacheManager = sparkSession.sharedState.cacheManager 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 c035f0d3b16ad..64317a04547a1 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 @@ -27,7 +27,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.{ResolvedPersistentView, ResolvedTable, ResolvedTempView} -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -309,8 +309,7 @@ case class DescribeRelationJsonCommand( catalog: SessionCatalog, metadata: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { - if (metadata.tableType == CatalogTableType.VIEW || - metadata.tableType == CatalogTableType.METRIC_VIEW) { + if (metadata.isViewLike) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(metadata.identifier.identifier) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 60bef37f4d3e5..fe3ec61ce0f11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -233,7 +233,7 @@ case class DropTableCommand( // issue an exception. catalog.getTableMetadata(tableName).tableType match { // Both VIEW and METRIC_VIEW are conceptually views and must be dropped via DROP VIEW. - case CatalogTableType.VIEW | CatalogTableType.METRIC_VIEW if !isView => + case t if CatalogTable.isViewLike(t) && !isView => throw QueryCompilationErrors.wrongCommandForObjectTypeError( operation = "DROP TABLE", requiredType = s"${CatalogTableType.EXTERNAL.name} or ${CatalogTableType.MANAGED.name}", @@ -241,7 +241,7 @@ case class DropTableCommand( foundType = catalog.getTableMetadata(tableName).tableType.name, alternative = "DROP VIEW" ) - case o if o != CatalogTableType.VIEW && o != CatalogTableType.METRIC_VIEW && isView => + case o if !CatalogTable.isViewLike(o) && isView => throw QueryCompilationErrors.wrongCommandForObjectTypeError( operation = "DROP VIEW", requiredType = @@ -1089,11 +1089,11 @@ object DDLUtils extends Logging { isView: Boolean): Unit = { if (!catalog.isTempView(tableMetadata.identifier)) { tableMetadata.tableType match { - case t if (t == CatalogTableType.VIEW || t == CatalogTableType.METRIC_VIEW) && !isView => + case t if CatalogTable.isViewLike(t) && !isView => throw QueryCompilationErrors.cannotAlterViewWithAlterTableError( viewName = tableMetadata.identifier.table ) - case o if o != CatalogTableType.VIEW && o != CatalogTableType.METRIC_VIEW && isView => + case o if !CatalogTable.isViewLike(o) && isView => throw QueryCompilationErrors.cannotAlterTableWithAlterViewError( tableName = tableMetadata.identifier.table ) 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 59068463497f9..ca534706635a1 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 @@ -103,8 +103,7 @@ case class CreateTableLikeCommand( provider } else if (fileFormat.inputFormat.isDefined) { Some(DDLUtils.HIVE_PROVIDER) - } else if (sourceTableDesc.tableType == CatalogTableType.VIEW || - sourceTableDesc.tableType == CatalogTableType.METRIC_VIEW) { + } else if (sourceTableDesc.isViewLike) { Some(sparkSession.sessionState.conf.defaultDataSourceName) } else { sourceTableDesc.provider @@ -268,8 +267,7 @@ case class AlterTableAddColumnsCommand( table: TableIdentifier): CatalogTable = { val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) - if (catalogTable.tableType == CatalogTableType.VIEW || - catalogTable.tableType == CatalogTableType.METRIC_VIEW) { + if (catalogTable.isViewLike) { throw QueryCompilationErrors.alterAddColNotSupportViewError(table) } @@ -732,8 +730,7 @@ case class DescribeTableCommand( catalog: SessionCatalog, metadata: CatalogTable, result: ArrayBuffer[Row]): Unit = { - if (metadata.tableType == CatalogTableType.VIEW || - metadata.tableType == CatalogTableType.METRIC_VIEW) { + if (metadata.isViewLike) { throw QueryCompilationErrors.descPartitionNotAllowedOnView(table.identifier) } DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 44b13fdcb3de9..ff6c1e067406d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -169,8 +169,7 @@ case class PreprocessTableCreation(catalog: SessionCatalog) extends Rule[Logical val tableName = tableIdentWithDB.unquotedString val existingTable = catalog.getTableMetadata(tableIdentWithDB) - if (existingTable.tableType == CatalogTableType.VIEW || - existingTable.tableType == CatalogTableType.METRIC_VIEW) { + if (existingTable.isViewLike) { throw QueryCompilationErrors.saveDataIntoViewNotAllowedError() } 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 69a01baa9f070..128fe57b6debf 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable +import scala.jdk.CollectionConverters._ import org.apache.hadoop.fs.Path @@ -350,8 +351,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // "no dependency list was supplied" while an empty list means "supplied but the // object has none". Metric-view CREATE always *computes* deps, so the right empty // representation is `Some(empty list)`, not `None`. - val deps = Some(DependencyList.of( - depParts.map(parts => Dependency.table(parts: _*)): _*)) + val sparkDeps: Seq[Dependency] = depParts.map(parts => Dependency.table(parts.asJava)) + val deps = Some(DependencyList.of(sparkDeps.asJava)) CreateV2MetricViewExec(viewCatalog, ident, userSpecifiedColumns, comment, mergedProps, originalText, analyzed, allowExisting, replace, deps) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 0d35380c61fdb..c268cd963b802 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -335,8 +335,7 @@ class V2SessionCatalog(catalog: SessionCatalog) try { loadTable(ident) match { case V1Table(v1Table) - if (v1Table.tableType == CatalogTableType.VIEW || - v1Table.tableType == CatalogTableType.METRIC_VIEW) && + if v1Table.isViewLike && !SQLConf.get.getConf(SQLConf.DROP_TABLE_VIEW_ENABLED) => throw QueryCompilationErrors.wrongCommandForObjectTypeError( operation = "DROP TABLE", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 3f5e8cd266c9f..ca387f6738fc8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -274,8 +274,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // Spark-created views do not have to be Hive compatible. If the data type is not // Hive compatible, we can set schema to empty so that Spark can still read this // view as the schema is also encoded in the table properties. - case schema if (tableDefinition.tableType == CatalogTableType.VIEW || - tableDefinition.tableType == CatalogTableType.METRIC_VIEW) && + case schema if tableDefinition.isViewLike && schema.exists(f => !isHiveCompatibleDataType(f.dataType)) => EMPTY_DATA_SCHEMA case other => other @@ -295,8 +294,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat try { client.createTable(tableWithDataSourceProps, ignoreIfExists) } catch { - case NonFatal(e) if (tableDefinition.tableType == CatalogTableType.VIEW || - tableDefinition.tableType == CatalogTableType.METRIC_VIEW) && + case NonFatal(e) if tableDefinition.isViewLike && hiveCompatibleSchema != EMPTY_DATA_SCHEMA => // If for some reason we fail to store the schema we store it as empty there // since we already store the real schema in the table properties. This try-catch diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index f376bc87841de..21db79116b52e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -1163,7 +1163,7 @@ private[hive] object HiveClientImpl extends Logging { catalogTableType match { case CatalogTableType.EXTERNAL => HiveTableType.EXTERNAL_TABLE case CatalogTableType.MANAGED => HiveTableType.MANAGED_TABLE - case CatalogTableType.VIEW | CatalogTableType.METRIC_VIEW => HiveTableType.VIRTUAL_VIEW + case t if CatalogTable.isViewLike(t) => HiveTableType.VIRTUAL_VIEW case t => throw new IllegalArgumentException( s"Unknown table type is found at toHiveTableType: $t") From f2251d80a2f446482642d9daea77539effd86b11 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Wed, 6 May 2026 14:03:41 +0000 Subject: [PATCH 15/17] [SPARK-XXXXX][SQL][FOLLOWUP] Switch Dependency API to arrays per cloud-fan Per https://github.com/apache/spark/pull/55487 review feedback, flip the Dependency / DependencyList records back to array-shape fields to match the dominant DSv2 convention (Identifier.namespace, Column.children, etc.) while preserving the value semantics that the round-3 List<> switch was introduced to fix. - TableDependency.nameParts: List -> String[] - FunctionDependency.nameParts: List -> String[] - DependencyList.dependencies: List -> Dependency[] - Each record gains explicit equals / hashCode / toString overrides using Arrays.equals / Arrays.hashCode / Arrays.toString so structural names still compare value-wise. Without the overrides, records' auto-generated methods on array fields fall through to Object.equals (reference equality). - Each record gains a defensive accessor override returning a clone, so callers cannot mutate the record's internal array. - Static factories Dependency.table / Dependency.function: switch from List to String[] (non-vararg, satisfying the same R3189677004 intent -- no `: _*` splat). - DependencyList.of: switch from List to Dependency[]. - DataSourceV2Strategy producer: build via .toArray instead of .asJava; drop the now-unused CollectionConverters import. - MetricViewV2CatalogSuite: array-shape access (.length / (0) / .toSeq) instead of .size / .get(0) / .asScala.toSeq. Also folds in three pre-existing CI failures from `90deee1949c`: - Dependency.java Javadoc: `{@link #function(String...)}` -> `{@link #function(String[])}` so `Run / Documentation generation` resolves the new signature. - MetricViewV2CatalogSuite ALTER VIEW RENAME TO test: drop the catalog component from the new name (`ns.mv_renamed`, not `testcat.ns.mv_renamed`) per upstream `DataSourceV2SQLSuite` convention, so `newName.asIdentifier` doesn't leak the catalog into the namespace and the fixture's relocated entry can be found by loadTableOrView. - PlanResolutionSuite `alter view: alter view properties`: stub the new `isViewLike` method on the Mockito CatalogTable mock at line 141 so Analyzer.scala:1229's call to `v1Table.v1Table.isViewLike` returns the correct value computed from the stubbed `tableType`. Without the stub Mockito returns false (default for unstubbed Boolean methods), the view fixture falls through to the table branch, and the test fails with `EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE`. Signed-off-by: chen.wang --- .../sql/connector/catalog/Dependency.java | 8 +-- .../sql/connector/catalog/DependencyList.java | 41 ++++++++++---- .../connector/catalog/FunctionDependency.java | 34 ++++++++--- .../connector/catalog/TableDependency.java | 34 ++++++++--- .../datasources/v2/DataSourceV2Strategy.scala | 6 +- .../execution/MetricViewV2CatalogSuite.scala | 56 ++++++++++--------- .../command/PlanResolutionSuite.scala | 6 ++ 7 files changed, 128 insertions(+), 57 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java index 6349dc1b05c48..4de02606b981f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Dependency.java @@ -17,8 +17,6 @@ package org.apache.spark.sql.connector.catalog; -import java.util.List; - import org.apache.spark.annotation.Evolving; /** @@ -29,7 +27,7 @@ *

* Note: today the only producer in Spark itself is metric-view dependency extraction, which * emits {@link TableDependency} only. {@link FunctionDependency} and the - * {@link #function(String...)} factory are exposed as groundwork for future producers + * {@link #function(String[])} factory are exposed as groundwork for future producers * (e.g. SQL UDF dependency tracking); consumers iterating a {@link DependencyList} received * from Spark today should expect to see only {@link TableDependency} instances. * @@ -44,7 +42,7 @@ public sealed interface Dependency permits TableDependency, FunctionDependency { * the first element is typically the catalog name and subsequent elements are namespace * components followed by the table name. */ - static TableDependency table(List nameParts) { + static TableDependency table(String[] nameParts) { return new TableDependency(nameParts); } @@ -54,7 +52,7 @@ static TableDependency table(List nameParts) { * catalog-managed functions the first element is typically the catalog name and subsequent * elements are namespace components followed by the function name. */ - static FunctionDependency function(List nameParts) { + static FunctionDependency function(String[] nameParts) { return new FunctionDependency(nameParts); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java index a790cfee91a05..9346bb364eba9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.connector.catalog; -import java.util.List; +import java.util.Arrays; import java.util.Objects; import org.apache.spark.annotation.Evolving; @@ -27,26 +27,47 @@ *

*

*

- * {@code dependencies} is held as an immutable {@link List} so the record's auto-generated - * {@code equals}/{@code hashCode} delegate to per-element value semantics on the contained - * {@link Dependency} entries. + * Records' auto-generated {@code equals}/{@code hashCode} on array fields fall through to + * {@link Object#equals} (reference equality), so this record overrides them to use + * {@link Arrays#equals(Object[], Object[])} / {@link Arrays#hashCode(Object[])} on + * {@code dependencies}; per-element equality delegates to the element's overridden + * {@code equals} ({@link TableDependency} / {@link FunctionDependency} both implement value + * semantics on their {@code nameParts} array). The defensive-copy accessor override clones + * on read so callers cannot mutate the record's internal array. * - * @param dependencies list of dependencies (immutable copy made) + * @param dependencies array of dependencies (defensive copy made) * @since 4.2.0 */ @Evolving -public record DependencyList(List dependencies) { +public record DependencyList(Dependency[] dependencies) { public DependencyList { Objects.requireNonNull(dependencies, "dependencies must not be null"); - dependencies = List.copyOf(dependencies); + dependencies = dependencies.clone(); } - public static DependencyList of(List dependencies) { + /** Returns a defensive copy of the underlying dependencies array. */ + @Override + public Dependency[] dependencies() { return dependencies.clone(); } + + @Override + public boolean equals(Object o) { + return o instanceof DependencyList that && Arrays.equals(dependencies, that.dependencies); + } + + @Override + public int hashCode() { return Arrays.hashCode(dependencies); } + + @Override + public String toString() { + return "DependencyList[dependencies=" + Arrays.toString(dependencies) + "]"; + } + + public static DependencyList of(Dependency[] dependencies) { return new DependencyList(dependencies); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java index f97f849b66353..0add00dc260eb 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.connector.catalog; -import java.util.List; +import java.util.Arrays; import java.util.Objects; import org.apache.spark.annotation.Evolving; @@ -28,19 +28,39 @@ * The dependent function is identified by its structural multi-part name. See * {@link TableDependency} for the parts-form contract. *

- * {@code nameParts} is held as an immutable {@link List} so the record's auto-generated - * {@code equals}/{@code hashCode} delegate to per-element value semantics. + * Records' auto-generated {@code equals}/{@code hashCode} on array fields fall through to + * {@link Object#equals} (reference equality), so this record overrides them to use + * {@link Arrays#equals(Object[], Object[])} / {@link Arrays#hashCode(Object[])} on + * {@code nameParts} and give value-based semantics. The defensive-copy accessor override + * also clones on read so callers cannot mutate the record's internal array. * - * @param nameParts structural multi-part identifier (immutable copy made; never empty) + * @param nameParts structural multi-part identifier (defensive copy made; never empty) * @since 4.2.0 */ @Evolving -public record FunctionDependency(List nameParts) implements Dependency { +public record FunctionDependency(String[] nameParts) implements Dependency { public FunctionDependency { Objects.requireNonNull(nameParts, "nameParts must not be null"); - if (nameParts.isEmpty()) { + if (nameParts.length == 0) { throw new IllegalArgumentException("nameParts must not be empty"); } - nameParts = List.copyOf(nameParts); + nameParts = nameParts.clone(); + } + + /** Returns a defensive copy of the underlying parts array. */ + @Override + public String[] nameParts() { return nameParts.clone(); } + + @Override + public boolean equals(Object o) { + return o instanceof FunctionDependency that && Arrays.equals(nameParts, that.nameParts); + } + + @Override + public int hashCode() { return Arrays.hashCode(nameParts); } + + @Override + public String toString() { + return "FunctionDependency[nameParts=" + Arrays.toString(nameParts) + "]"; } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java index 717a6d9f61091..a1e5966e15d18 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.connector.catalog; -import java.util.List; +import java.util.Arrays; import java.util.Objects; import org.apache.spark.annotation.Evolving; @@ -36,19 +36,39 @@ * string should join the parts themselves with a quoting scheme appropriate to their wire * format. *

- * {@code nameParts} is held as an immutable {@link List} so the record's auto-generated - * {@code equals}/{@code hashCode} delegate to per-element value semantics. + * Records' auto-generated {@code equals}/{@code hashCode} on array fields fall through to + * {@link Object#equals} (reference equality), so this record overrides them to use + * {@link Arrays#equals(Object[], Object[])} / {@link Arrays#hashCode(Object[])} on + * {@code nameParts} and give value-based semantics. The defensive-copy accessor override + * also clones on read so callers cannot mutate the record's internal array. * - * @param nameParts structural multi-part identifier (immutable copy made; never empty) + * @param nameParts structural multi-part identifier (defensive copy made; never empty) * @since 4.2.0 */ @Evolving -public record TableDependency(List nameParts) implements Dependency { +public record TableDependency(String[] nameParts) implements Dependency { public TableDependency { Objects.requireNonNull(nameParts, "nameParts must not be null"); - if (nameParts.isEmpty()) { + if (nameParts.length == 0) { throw new IllegalArgumentException("nameParts must not be empty"); } - nameParts = List.copyOf(nameParts); + nameParts = nameParts.clone(); + } + + /** Returns a defensive copy of the underlying parts array. */ + @Override + public String[] nameParts() { return nameParts.clone(); } + + @Override + public boolean equals(Object o) { + return o instanceof TableDependency that && Arrays.equals(nameParts, that.nameParts); + } + + @Override + public int hashCode() { return Arrays.hashCode(nameParts); } + + @Override + public String toString() { + return "TableDependency[nameParts=" + Arrays.toString(nameParts) + "]"; } } 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 128fe57b6debf..e113475811092 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 @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable -import scala.jdk.CollectionConverters._ import org.apache.hadoop.fs.Path @@ -351,8 +350,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // "no dependency list was supplied" while an empty list means "supplied but the // object has none". Metric-view CREATE always *computes* deps, so the right empty // representation is `Some(empty list)`, not `None`. - val sparkDeps: Seq[Dependency] = depParts.map(parts => Dependency.table(parts.asJava)) - val deps = Some(DependencyList.of(sparkDeps.asJava)) + val sparkDeps: Array[Dependency] = + depParts.map(parts => Dependency.table(parts.toArray): Dependency).toArray + val deps = Some(DependencyList.of(sparkDeps)) CreateV2MetricViewExec(viewCatalog, ident, userSpecifiedColumns, comment, mergedProps, originalText, analyzed, allowExisting, replace, deps) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala index aa8be6dab6493..54caafe7b5f15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetricViewV2CatalogSuite.scala @@ -153,9 +153,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val deps = info.viewDependencies() assert(deps != null) - assert(deps.dependencies().size() === 1) - val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().asScala.toSeq === + assert(deps.dependencies().length === 1) + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -210,9 +210,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { assert(props.get(TableCatalog.PROP_COMMENT) === "my mv") val deps = info.viewDependencies() - assert(deps != null && deps.dependencies().size() === 1) - val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().asScala.toSeq === + assert(deps != null && deps.dependencies().length === 1) + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -314,9 +314,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { assert(finalInfo.queryText().trim === replacementYaml.trim) assert(finalInfo.properties().get(MetricView.PROP_WHERE) === "count > 100") val deps = finalInfo.viewDependencies() - assert(deps != null && deps.dependencies().size() === 1) - val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().asScala.toSeq === + assert(deps != null && deps.dependencies().length === 1) + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -506,8 +506,8 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { val deps = capturedViewInfo().viewDependencies() assert(deps != null) - val depParts = deps.dependencies().asScala - .map(_.asInstanceOf[TableDependency].nameParts().asScala.toSeq).toSet + val depParts = deps.dependencies() + .map(_.asInstanceOf[TableDependency].nameParts().toSeq).toSet assert(depParts === Set( Seq(testCatalogName, testNamespace, sourceTableName), Seq(testCatalogName, testNamespace, "customers")), @@ -531,11 +531,11 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, metricView) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().size() === 1, + assert(deps != null && deps.dependencies().length === 1, s"Expected 1 deduplicated dependency, got " + - s"${Option(deps).map(_.dependencies().size()).getOrElse(0)}") - val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().asScala.toSeq === + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -556,11 +556,11 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, metricView) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().size() === 1, + assert(deps != null && deps.dependencies().length === 1, s"Expected 1 deduplicated dependency for self-join, got " + - s"${Option(deps).map(_.dependencies().size()).getOrElse(0)}") - val tableDep = deps.dependencies().get(0).asInstanceOf[TableDependency] - assert(tableDep.nameParts().asScala.toSeq === + s"${Option(deps).map(_.dependencies().length).getOrElse(0)}") + val tableDep = deps.dependencies()(0).asInstanceOf[TableDependency] + assert(tableDep.nameParts().toSeq === Seq(testCatalogName, testNamespace, sourceTableName)) } } @@ -581,9 +581,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, mv) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().size() === 1) + assert(deps != null && deps.dependencies().length === 1) val parts = - deps.dependencies().get(0).asInstanceOf[TableDependency].nameParts().asScala.toSeq + deps.dependencies()(0).asInstanceOf[TableDependency].nameParts().toSeq // `MetricViewHelper.qualifyV1` normalizes any `TableIdentifier.nameParts` shape // (1, 2, or 3 parts depending on what the analyzer captured) to the stable // `[spark_catalog, db, table]` shape so downstream consumers see deterministic @@ -618,9 +618,9 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { createMetricView(fullMetricViewName, mv) val deps = capturedViewInfo().viewDependencies() - assert(deps != null && deps.dependencies().size() === 1) + assert(deps != null && deps.dependencies().length === 1) val parts = - deps.dependencies().get(0).asInstanceOf[TableDependency].nameParts().asScala.toSeq + deps.dependencies()(0).asInstanceOf[TableDependency].nameParts().toSeq assert(parts === Seq(testCatalogName, multiNamespace(0), multiNamespace(1), multiTable), s"Multi-level nameParts should preserve every namespace component, got $parts") } finally { @@ -892,14 +892,20 @@ class MetricViewV2CatalogSuite extends QueryTest with SharedSparkSession { where = None, select = metricViewColumns) createMetricView(fullMetricViewName, mv) - val renamedFull = s"$testCatalogName.$testNamespace.mv_renamed" + // Per upstream DataSourceV2SQLSuite convention (see lines 2477 / 2484 there), the + // RENAME TO clause takes a 2-part `namespace.name` -- the new ident is implicitly + // within the same catalog as the source view. Including a 3-part `catalog.ns.name` + // would leak the catalog component into `newName.asIdentifier` and the catalog's + // `renameView` would store under a key the loader can't find. + val renamedRelative = s"$testNamespace.mv_renamed" + val renamedFull = s"$testCatalogName.$renamedRelative" try { // RenameTable on a `ResolvedPersistentView` is routed by `DataSourceV2Strategy` to // `RenameV2ViewExec`, which calls `ViewCatalog.renameView` -- the fixture // `MetricViewRecordingCatalog.renameView` relocates both the `views` entry and the // `capturedViews` entry under the new ident. Pin the wiring end-to-end so the // metric view kind survives the rename. - sql(s"ALTER VIEW $fullMetricViewName RENAME TO $renamedFull") + sql(s"ALTER VIEW $fullMetricViewName RENAME TO $renamedRelative") // Old ident is gone from the v2 catalog -- DESCRIBE should fail to resolve. val oldEx = intercept[AnalysisException] { 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 d31f69dcfdd8d..907aa895a562b 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 @@ -139,6 +139,12 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { .add("s", "string") .add("point", new StructType().add("x", "int").add("y", "int"))) when(t.tableType).thenReturn(tableType) + // Mockito returns false for unstubbed Boolean methods, so analyzer code paths that + // dispatch through `CatalogTable.isViewLike` (e.g. `Analyzer.lookupTableOrView`'s v1 + // session-catalog branch) would misclassify a mocked VIEW fixture as a table. Stub + // the method to compute from the just-stubbed `tableType` so any view-like type + // (VIEW today, METRIC_VIEW or future kinds) resolves correctly. + when(t.isViewLike).thenReturn(CatalogTable.isViewLike(tableType)) when(t.provider).thenReturn(Some(provider)) when(t.identifier).thenReturn( ident.asTableIdentifier.copy(catalog = Some(SESSION_CATALOG_NAME))) From ed2db225a066f9ca88d31276777d4fc391dbee99 Mon Sep 17 00:00:00 2001 From: "chen.wang" Date: Wed, 6 May 2026 17:20:08 +0000 Subject: [PATCH 16/17] [SPARK-XXXXX][SQL][FOLLOWUP] Address cloud-fan round 6 review on PR #55487 Address 4 findings from https://github.com/apache/spark/pull/55487#pullrequestreview-4237214958: - Drop "DBR" terminology leak in CatalogTable.isViewLike Javadoc (interface.scala:466) -- newly introduced in round 5; matches the resolution of round 4's DBR-mention finding on test comments. - Drop misleading "Body kept in sync with the instance method" line in the CatalogTable.isViewLike(t: CatalogTableType) companion Javadoc (interface.scala:773-774) -- there's only one body since the instance method delegates to the companion form. - Document the non-null-element precondition on TableDependency, FunctionDependency, and DependencyList @param Javadocs. Round-5b's switch from List.copyOf back to Array.clone() lost the implicit null-element rejection that the immutable-list invariant gave. Spark's own producer never emits nulls, but the SPI is @Evolving and exposed to third-party catalogs; documenting the precondition keeps the API consistent with other DSv2 array-shape SPIs (Identifier.namespace, Column.children) which also trust callers. - views.scala:175 (CreateViewCommand.runCommand) audit miss: change `tableMetadata.tableType != CatalogTableType.VIEW` to `!tableMetadata.isViewLike` so `CREATE VIEW ` surfaces the correct viewAlreadyExistsError instead of the misclassifying unsupportedCreateOrReplaceViewOnTableError ("...on a table"). Pre-PR this branch didn't fire on metric views (they had tableType == VIEW); post-PR they have tableType == METRIC_VIEW, so the strict comparison incorrectly classified them as tables. Also widen the two assertions in createDataSourceTables.scala (`assert(table.tableType != CatalogTableType.VIEW)`) to `assert(!table.isViewLike)` per the same pattern -- these are invariant sanity checks whose intent is "no view-like input here". Signed-off-by: chen.wang --- .../spark/sql/connector/catalog/DependencyList.java | 4 +++- .../spark/sql/connector/catalog/FunctionDependency.java | 4 +++- .../spark/sql/connector/catalog/TableDependency.java | 4 +++- .../apache/spark/sql/catalyst/catalog/interface.scala | 9 ++++----- .../sql/execution/command/createDataSourceTables.scala | 4 ++-- .../org/apache/spark/sql/execution/command/views.scala | 2 +- 6 files changed, 16 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java index 9346bb364eba9..21c1e662fda62 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DependencyList.java @@ -39,7 +39,9 @@ * semantics on their {@code nameParts} array). The defensive-copy accessor override clones * on read so callers cannot mutate the record's internal array. * - * @param dependencies array of dependencies (defensive copy made) + * @param dependencies array of dependencies; must contain no null elements (defensive + * copy made; not validated element-wise -- callers passing nulls will + * surface NPEs in downstream consumers) * @since 4.2.0 */ @Evolving diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java index 0add00dc260eb..c19d118043afa 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/FunctionDependency.java @@ -34,7 +34,9 @@ * {@code nameParts} and give value-based semantics. The defensive-copy accessor override * also clones on read so callers cannot mutate the record's internal array. * - * @param nameParts structural multi-part identifier (defensive copy made; never empty) + * @param nameParts structural multi-part identifier; must be non-empty and contain no + * null elements (defensive copy made; not validated element-wise -- + * callers passing nulls will surface NPEs in downstream consumers) * @since 4.2.0 */ @Evolving diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java index a1e5966e15d18..0aa9a47311607 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableDependency.java @@ -42,7 +42,9 @@ * {@code nameParts} and give value-based semantics. The defensive-copy accessor override * also clones on read so callers cannot mutate the record's internal array. * - * @param nameParts structural multi-part identifier (defensive copy made; never empty) + * @param nameParts structural multi-part identifier; must be non-empty and contain no + * null elements (defensive copy made; not validated element-wise -- + * callers passing nulls will surface NPEs in downstream consumers) * @since 4.2.0 */ @Evolving 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 ff390f14fcd3e..efd4bf621921e 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 @@ -463,9 +463,9 @@ case class CatalogTable( /** * Returns whether this table behaves like a view at resolution / DDL time. Today: VIEW or - * METRIC_VIEW. Forks may extend this set (e.g. DBR also includes MATERIALIZED_VIEW and - * STREAMING_TABLE), so call sites that need a uniform "is this view-like?" check should - * prefer this helper over inline disjunctions on `tableType`. + * METRIC_VIEW. Forks may extend this set with additional view-like types, so call sites + * that need a uniform "is this view-like?" check should prefer this helper over inline + * disjunctions on `tableType`. */ def isViewLike: Boolean = CatalogTable.isViewLike(tableType) @@ -770,8 +770,7 @@ object CatalogTable { * Type-only form of [[CatalogTable.isViewLike]]; returns whether the given table type * behaves like a view at resolution / DDL time. Use this overload when you have a * [[CatalogTableType]] but no surrounding [[CatalogTable]] (e.g. inside `match`/`case` - * patterns or [[org.apache.spark.sql.catalyst.catalog.SessionCatalog.isView]]). Body kept - * in sync with the instance method. + * patterns or [[org.apache.spark.sql.catalyst.catalog.SessionCatalog.isView]]). */ def isViewLike(tableType: CatalogTableType): Boolean = { tableType == CatalogTableType.VIEW || tableType == CatalogTableType.METRIC_VIEW diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 0415a33e2d6dd..7519216f1b367 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -50,7 +50,7 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo extends LeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - assert(table.tableType != CatalogTableType.VIEW) + assert(!table.isViewLike) assert(table.provider.isDefined) val sessionState = sparkSession.sessionState @@ -151,7 +151,7 @@ case class CreateDataSourceTableAsSelectCommand( override def innerChildren: Seq[LogicalPlan] = query :: Nil override def run(sparkSession: SparkSession): Seq[Row] = { - assert(table.tableType != CatalogTableType.VIEW) + assert(!table.isViewLike) assert(table.provider.isDefined) val sessionState = sparkSession.sessionState 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 098cc35a63f83..8407f20777d98 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 @@ -172,7 +172,7 @@ case class CreateViewCommand( if (allowExisting) { // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view // already exists. - } else if (tableMetadata.tableType != CatalogTableType.VIEW) { + } else if (!tableMetadata.isViewLike) { throw QueryCompilationErrors.unsupportedCreateOrReplaceViewOnTableError( name.nameParts, replace) } else if (replace) { From f22b27d83d64e6a16165157c95355a7e4a33ae45 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 7 May 2026 03:03:47 +0800 Subject: [PATCH 17/17] Apply suggestions from code review Co-authored-by: Wenchen Fan --- .../scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index ca387f6738fc8..5d3a872d047c1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -602,7 +602,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat requireTableExists(db, tableDefinition.identifier.table) verifyTableProperties(tableDefinition) - if (tableDefinition.tableType == VIEW || tableDefinition.tableType == METRIC_VIEW) { + if (tableDefinition.isViewLike) { val newTableProps = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition).toMap val schemaWithNoCollation = removeCollation(tableDefinition.schema) val hiveCompatibleSchema = @@ -851,7 +851,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } table.properties.get(DATASOURCE_PROVIDER) match { - case None if table.tableType == VIEW || table.tableType == METRIC_VIEW => + case None if table.isViewLike => // If this is a view created by Spark 2.2 or higher versions, we should restore its schema // from table properties. getSchemaFromTableProperties(table.properties).foreach { schemaFromTableProps =>