From db13658d1182f770b1d4487bfa16a72b44ffd808 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Fri, 22 Jul 2022 17:30:45 -0700 Subject: [PATCH 01/26] initial parser support --- docs/sql-ref-ansi-compliance.md | 2 ++ .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 2 ++ .../sql/catalyst/parser/SqlBaseParser.g4 | 10 ++++++- .../sql/catalyst/parser/AstBuilder.scala | 29 +++++++++++++++---- .../sql/catalyst/parser/DDLParserSuite.scala | 19 ++++++++++++ 5 files changed, 56 insertions(+), 6 deletions(-) diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 65ed5caf83344..5367e7b32e0a2 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -344,6 +344,7 @@ Below is a list of all the keywords in Spark SQL. |AFTER|non-reserved|non-reserved|non-reserved| |ALL|reserved|non-reserved|reserved| |ALTER|non-reserved|non-reserved|reserved| +|ALWAYS|non-reserved|non-reserved|non-reserved| |ANALYZE|non-reserved|non-reserved|non-reserved| |AND|reserved|non-reserved|reserved| |ANTI|non-reserved|strict-non-reserved|non-reserved| @@ -440,6 +441,7 @@ Below is a list of all the keywords in Spark SQL. |FULL|reserved|strict-non-reserved|reserved| |FUNCTION|non-reserved|non-reserved|reserved| |FUNCTIONS|non-reserved|non-reserved|non-reserved| +|GENERATED|non-reserved|non-reserved|non-reserved| |GLOBAL|non-reserved|non-reserved|reserved| |GRANT|reserved|non-reserved|reserved| |GROUP|reserved|non-reserved|reserved| diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 1cbd6d24deaec..95f37f7cf77f8 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -91,6 +91,7 @@ ADD: 'ADD'; AFTER: 'AFTER'; ALL: 'ALL'; ALTER: 'ALTER'; +ALWAYS: 'ALWAYS'; ANALYZE: 'ANALYZE'; AND: 'AND'; ANTI: 'ANTI'; @@ -187,6 +188,7 @@ FROM: 'FROM'; FULL: 'FULL'; FUNCTION: 'FUNCTION'; FUNCTIONS: 'FUNCTIONS'; +GENERATED: 'GENERATED'; GLOBAL: 'GLOBAL'; GRANT: 'GRANT'; GROUP: 'GROUP'; diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index f398ddd76f712..727f77b367943 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -942,7 +942,11 @@ createOrReplaceTableColTypeList ; createOrReplaceTableColType - : colName=errorCapturingIdentifier dataType (NOT NULL)? defaultExpression? commentSpec? + : colName=errorCapturingIdentifier dataType (NOT NULL)? defaultExpression? generationExpression? commentSpec? + ; + +generationExpression + : GENERATED ALWAYS AS LEFT_PAREN expression RIGHT_PAREN ; complexColTypeList @@ -1072,6 +1076,7 @@ ansiNonReserved : ADD | AFTER | ALTER + | ALWAYS | ANALYZE | ANTI | ANY_VALUE @@ -1139,6 +1144,7 @@ ansiNonReserved | FORMATTED | FUNCTION | FUNCTIONS + | GENERATED | GLOBAL | GROUPING | HOUR @@ -1314,6 +1320,7 @@ nonReserved | AFTER | ALL | ALTER + | ALWAYS | ANALYZE | AND | ANY @@ -1406,6 +1413,7 @@ nonReserved | FROM | FUNCTION | FUNCTIONS + | GENERATED | GLOBAL | GRANT | GROUP diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 890c8f4000f5e..7b2057d84d831 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2816,6 +2816,13 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) } } + // Add the 'GENERATED ALWAYS AS expression' clause in the column definition, if any, to the + // column metadata. + Option(ctx.generationExpression()).map(visitGenerationExpression).foreach { field => + // TODO: check that generated columns is enabled + // TODO: where do we store this key + builder.putString("generationExpression", field) + } val name: String = colName.getText @@ -2874,11 +2881,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit string(ctx.STRING) } - /** - * Create a default string. - */ - override def visitDefaultExpression(ctx: DefaultExpressionContext): String = withOrigin(ctx) { - val exprCtx = ctx.expression() + private def verifyAndGetExpression(exprCtx: ExpressionContext): String = { // Make sure it can be converted to Catalyst expressions. expression(exprCtx) // Extract the raw expression text so that we can save the user provided text. We don't @@ -2890,6 +2893,22 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit exprCtx.getStart.getInputStream.getText(new Interval(start, end)) } + /** + * Create a default string. + */ + override def visitDefaultExpression(ctx: DefaultExpressionContext): String = + withOrigin(ctx) { + verifyAndGetExpression(ctx.expression()) + } + + /** + * Create a generation expression string. + */ + override def visitGenerationExpression(ctx: GenerationExpressionContext): String = + withOrigin(ctx) { + verifyAndGetExpression(ctx.expression()) + } + /** * Create an optional comment string. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index eb3e9baaacd55..4032ecec76f67 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2320,4 +2320,23 @@ class DDLParserSuite extends AnalysisTest { Seq(Assignment(UnresolvedAttribute("target.col1"), UnresolvedAttribute("source.col1")), Assignment(UnresolvedAttribute("target.col2"), UnresolvedAttribute("DEFAULT"))))))) } + + test("Implement parser support for GENERATED ALWAYS AS columns in tables") { + val schemaWithGeneratedColumn = new StructType() + .add("a", IntegerType, true) + .add("b", IntegerType, false, + new MetadataBuilder().putString("generationExpression", "a+1").build()) + comparePlans(parsePlan( + "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), + CreateTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithGeneratedColumn, + Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), + Map.empty[String, String], None, None, None, false), false)) + comparePlans(parsePlan( + "REPLACE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), + ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithGeneratedColumn, + Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), + Map.empty[String, String], None, None, None, false), false)) + + // TODO: test error when feature is disabled + } } From 29d11ec51b8930b8f9ead2f54a3f23f3dddce27e Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Tue, 2 Aug 2022 19:21:09 -0700 Subject: [PATCH 02/26] add sql conf --- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 9 ++++++--- .../org/apache/spark/sql/errors/QueryParsingErrors.scala | 4 ++++ .../scala/org/apache/spark/sql/internal/SQLConf.scala | 9 +++++++++ .../spark/sql/catalyst/parser/DDLParserSuite.scala | 8 ++++++-- 4 files changed, 25 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 7b2057d84d831..8e263c46f4652 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2819,9 +2819,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit // Add the 'GENERATED ALWAYS AS expression' clause in the column definition, if any, to the // column metadata. Option(ctx.generationExpression()).map(visitGenerationExpression).foreach { field => - // TODO: check that generated columns is enabled - // TODO: where do we store this key - builder.putString("generationExpression", field) + if (conf.getConf(SQLConf.ENABLE_GENERATED_COLUMNS)) { + // TODO: where do we store this key + builder.putString("generationExpression", field) + } else { + throw QueryParsingErrors.generatedColumnNotEnabledError(ctx) + } } val name: String = colName.getText diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index d4629f0dd3fe4..0357ec528356e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -540,4 +540,8 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( "References to DEFAULT column values are not allowed within the PARTITION clause", ctx) } + + def generatedColumnNotEnabledError(ctx: ParserRuleContext): Throwable = { + new ParseException("Support for GENERATED ALWAYS AS column is not allowed", ctx) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1b7857ead59fa..4f6daf2628972 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2922,6 +2922,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val ENABLE_GENERATED_COLUMNS = + buildConf("spark.sql.generatedColumn.enabled") + .internal() + .doc("When true, allow CREATE TABLE statements to specify a generation expression for" + + "specific columns.") + .version("3.4.0") + .booleanConf + .createWithDefault(true) + val ENFORCE_RESERVED_KEYWORDS = buildConf("spark.sql.ansi.enforceReservedKeywords") .doc(s"When true and '${ANSI_ENABLED.key}' is true, the Spark SQL parser enforces the ANSI " + "reserved keywords and forbids SQL queries that use reserved keywords as alias names " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 4032ecec76f67..c7f6e4452199e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2336,7 +2336,11 @@ class DDLParserSuite extends AnalysisTest { ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithGeneratedColumn, Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), Map.empty[String, String], None, None, None, false), false)) - - // TODO: test error when feature is disabled + // Make sure that the parser returns an exception when the feature is disabled. + withSQLConf(SQLConf.ENABLE_GENERATED_COLUMNS.key -> "false") { + intercept( + "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet", + "Support for GENERATED ALWAYS AS column is not allowed") + } } } From 3c19862d300bcfcf39df7f76f0b6094b856df3e4 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Sun, 27 Nov 2022 21:01:55 -0800 Subject: [PATCH 03/26] table provider support --- .../sql/connector/catalog/TableProvider.java | 7 ++++++ .../sql/errors/QueryCompilationErrors.scala | 9 +++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 1 + .../analysis/ResolveSessionCatalog.scala | 20 ++++++++++++++- .../sql/connector/DataSourceV2Suite.scala | 25 +++++++++++++++++++ 5 files changed, 61 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index 6b68fbe123019..e144536f901bf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -93,4 +93,11 @@ default Transform[] inferPartitioning(CaseInsensitiveStringMap options) { default boolean supportsExternalMetadata() { return false; } + + /** + * TODO + */ + default boolean supportsGeneratedColumnsOnCreation() { + return false; + } } 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 c828318f2cdd3..fcf8f76e75b4e 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 @@ -2528,4 +2528,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { errorClass = "INVALID_COLUMN_OR_FIELD_DATA_TYPE", messageParameters = Array(toSQLId(name), toSQLType(dt), toSQLType(expected))) } + + def generatedColumnsNotAllowedInDataSource(provider: String): Throwable = { + new AnalysisException( + s""" + |Failed to execute command because GENERATED ALWAYS AS expressions are not supported for + |target data source with table provider: \"$provider\" + |""".stripMargin + ) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index c7f6e4452199e..a5d2adee3fba5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2342,5 +2342,6 @@ class DDLParserSuite extends AnalysisTest { "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet", "Support for GENERATED ALWAYS AS column is not allowed") } + // TODO: add test for non sql expression string } } 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 a495e35bf2c2e..2f1c80e8fa177 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultCols} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, Identifier, LookupCatalog, SupportsNamespaces, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, Identifier, LookupCatalog, SupportsNamespaces, TableProvider, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command._ @@ -156,6 +156,14 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) val (storageFormat, provider) = getStorageFormatAndProvider( c.tableSpec.provider, c.tableSpec.options, c.tableSpec.location, c.tableSpec.serde, ctas = false) + + // todo: move these into a GeneratedColumn module? + def isGeneratedColumn(field: StructField) = field.metadata.contains("generationExpression") + val hasGeneratedColumn = c.tableSchema.exists(isGeneratedColumn) + + if (hasGeneratedColumn && !supportsGeneratedColumnsOnCreation(provider)) { + throw QueryCompilationErrors.generatedColumnsNotAllowedInDataSource(provider) + } if (!isV2Provider(provider)) { constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, c.ignoreIfExists, storageFormat, provider) @@ -614,6 +622,16 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } } + private def supportsGeneratedColumnsOnCreation(provider: String): Boolean = { + // Return earlier since `lookupDataSourceV2` may fail to resolve provider "hive" to + // `HiveFileFormat`, when running tests in sql/core. + if (DDLUtils.isHiveTable(Some(provider))) return false + DataSource.lookupDataSource(provider, conf).newInstance() match { + case t: TableProvider => t.supportsGeneratedColumnsOnCreation() + case _ => false + } + } + private object DatabaseInSessionCatalog { def unapply(resolved: ResolvedNamespace): Option[String] = resolved match { case ResolvedNamespace(catalog, _) if !isSessionCatalog(catalog) => None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 5c4be75e02c7f..ab4d9c2db9896 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -596,6 +596,27 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS } } } + + test("SPARK-41290: cannot CREATE TABLE with GENERATED ALWAYS AS if table provider does not " + + "support generated columns on creation") { + val supportsGeneratedColumn = + classOf[SupportsGeneratedColumnsOnCreationWritableDataSource].getName + val doesNotSupportGeneratedColumn = classOf[SimpleWritableDataSource].getName + val tblName = "my_tab" + withTable(tblName) { + spark.sql(s"CREATE TABLE my_tab_1(a INT, b INT GENERATED ALWAYS AS (a+1)) " + + s"USING $supportsGeneratedColumn") + } + withTable(tblName) { + val e = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE my_tab_2(a INT, b INT GENERATED ALWAYS AS (a+1)) " + + s"USING $doesNotSupportGeneratedColumn") + } + assert(e.getMessage.contains("GENERATED ALWAYS AS expressions are not supported for\n" + + "target data source with table provider: " + + "\"org.apache.spark.sql.connector.SimpleWritableDataSource\"")) + } + } } @@ -1106,3 +1127,7 @@ class ReportStatisticsDataSource extends SimpleWritableDataSource { } } } + +class SupportsGeneratedColumnsOnCreationWritableDataSource extends SimpleWritableDataSource { + override def supportsGeneratedColumnsOnCreation(): Boolean = true +} From ebc5ed94ec94cd52ad18ad40539c9871456cf034 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Sun, 27 Nov 2022 22:41:51 -0800 Subject: [PATCH 04/26] refactor --- .../sql/connector/catalog/TableProvider.java | 9 +++- .../sql/catalyst/parser/AstBuilder.scala | 5 +-- .../sql/catalyst/util/GeneratedColumn.scala | 43 +++++++++++++++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 3 +- .../analysis/ResolveSessionCatalog.scala | 9 ++-- 5 files changed, 57 insertions(+), 12 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index e144536f901bf..87b8c1fd40e91 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -95,7 +95,14 @@ default boolean supportsExternalMetadata() { } /** - * TODO + * Returns true if the source supports defining generated columns upon table creation in SQL. + * When false any create table statements with a generated column defined in the table schema will + * throw an exception during analysis. + * + * A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} + * The generation expression is stored in the column metadata with key "generationExpression". + * + * Override this method to allow defining generated columns in create table statements. */ default boolean supportsGeneratedColumnsOnCreation() { return false; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 8e263c46f4652..60ce56d02d927 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils, ResolveDefaultColumns} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, GeneratedColumn, IntervalUtils, ResolveDefaultColumns} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTimestamp, stringToTimestampWithoutTimeZone} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition @@ -2820,8 +2820,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit // column metadata. Option(ctx.generationExpression()).map(visitGenerationExpression).foreach { field => if (conf.getConf(SQLConf.ENABLE_GENERATED_COLUMNS)) { - // TODO: where do we store this key - builder.putString("generationExpression", field) + builder.putString(GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY, field) } else { throw QueryParsingErrors.generatedColumnNotEnabledError(ctx) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala new file mode 100644 index 0000000000000..ab19184683261 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.apache.spark.sql.types.{StructField, StructType} + +/** + * This object contains utility methods and values for Generated Columns + */ +object GeneratedColumn { + + /** The metadata key for saving a generation expression in a generated column's metadata */ + val GENERATION_EXPRESSION_METADATA_KEY = "generationExpression" + + /** + * Whether the given `field` is a generated column + */ + private def isGeneratedColumn(field: StructField): Boolean = { + field.metadata.contains(GENERATION_EXPRESSION_METADATA_KEY) + } + + /** + * Whether the `schema` has one or more generated columns + */ + def hasGeneratedColumns(schema: StructType): Boolean = { + schema.exists(isGeneratedColumn) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index a5d2adee3fba5..144a9ca29fb85 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2321,7 +2321,7 @@ class DDLParserSuite extends AnalysisTest { Assignment(UnresolvedAttribute("target.col2"), UnresolvedAttribute("DEFAULT"))))))) } - test("Implement parser support for GENERATED ALWAYS AS columns in tables") { + test("SPARK-41290: implement parser support for GENERATED ALWAYS AS columns in tables") { val schemaWithGeneratedColumn = new StructType() .add("a", IntegerType, true) .add("b", IntegerType, false, @@ -2342,6 +2342,5 @@ class DDLParserSuite extends AnalysisTest { "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet", "Support for GENERATED ALWAYS AS column is not allowed") } - // TODO: add test for non sql expression string } } 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 2f1c80e8fa177..3e91d7b29372a 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultCols} +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, GeneratedColumn, ResolveDefaultColumns => DefaultCols} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, Identifier, LookupCatalog, SupportsNamespaces, TableProvider, V1Table} import org.apache.spark.sql.connector.expressions.Transform @@ -157,11 +157,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) c.tableSpec.provider, c.tableSpec.options, c.tableSpec.location, c.tableSpec.serde, ctas = false) - // todo: move these into a GeneratedColumn module? - def isGeneratedColumn(field: StructField) = field.metadata.contains("generationExpression") - val hasGeneratedColumn = c.tableSchema.exists(isGeneratedColumn) - - if (hasGeneratedColumn && !supportsGeneratedColumnsOnCreation(provider)) { + if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && + !supportsGeneratedColumnsOnCreation(provider)) { throw QueryCompilationErrors.generatedColumnsNotAllowedInDataSource(provider) } if (!isV2Provider(provider)) { From 02f578399965a411bc05bdec7bb5441c402fafda Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Sun, 27 Nov 2022 23:17:19 -0800 Subject: [PATCH 05/26] check for replace statements --- .../analysis/ResolveSessionCatalog.scala | 5 ++++ .../sql/connector/DataSourceV2Suite.scala | 26 +++++++++++-------- 2 files changed, 20 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 3e91d7b29372a..2bdd23abd1a2f 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 @@ -195,6 +195,11 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case c @ ReplaceTable(ResolvedIdentifier(catalog, _), _, _, _, _) if isSessionCatalog(catalog) => val provider = c.tableSpec.provider.getOrElse(conf.defaultDataSourceName) + + if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && + !supportsGeneratedColumnsOnCreation(provider)) { + throw QueryCompilationErrors.generatedColumnsNotAllowedInDataSource(provider) + } if (!isV2Provider(provider)) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError("REPLACE TABLE") } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index ab4d9c2db9896..4c6a9dfda4e3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -603,18 +603,22 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS classOf[SupportsGeneratedColumnsOnCreationWritableDataSource].getName val doesNotSupportGeneratedColumn = classOf[SimpleWritableDataSource].getName val tblName = "my_tab" - withTable(tblName) { - spark.sql(s"CREATE TABLE my_tab_1(a INT, b INT GENERATED ALWAYS AS (a+1)) " + - s"USING $supportsGeneratedColumn") - } - withTable(tblName) { - val e = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE my_tab_2(a INT, b INT GENERATED ALWAYS AS (a+1)) " + - s"USING $doesNotSupportGeneratedColumn") + val tableDefinition = s"TABLE $tblName(a INT, b INT GENERATED ALWAYS AS (a+1))" + for (statement <- Seq("CREATE", "REPLACE")) { + withTable(tblName) { + if (statement == "REPLACE") { + spark.sql(s"CREATE TABLE $tblName(a INT) USING $supportsGeneratedColumn") + } + spark.sql(s"$statement $tableDefinition USING $supportsGeneratedColumn") + } + withTable(tblName) { + val e = intercept[AnalysisException] { + spark.sql(s"$statement $tableDefinition USING $doesNotSupportGeneratedColumn") + } + assert(e.getMessage.contains("GENERATED ALWAYS AS expressions are not supported for\n" + + "target data source with table provider: " + + "\"org.apache.spark.sql.connector.SimpleWritableDataSource\"")) } - assert(e.getMessage.contains("GENERATED ALWAYS AS expressions are not supported for\n" + - "target data source with table provider: " + - "\"org.apache.spark.sql.connector.SimpleWritableDataSource\"")) } } } From f9cfd6bade0b2a3a707ee147befafeb637f5a611 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Mon, 28 Nov 2022 12:02:54 -0800 Subject: [PATCH 06/26] fix updates from master --- core/src/main/resources/error/error-classes.json | 5 +++++ .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 10 +++++++++- .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../spark/sql/catalyst/parser/DDLParserSuite.scala | 13 ++++++++++--- 5 files changed, 26 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 897287772011e..454bdb88fc20a 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1803,6 +1803,11 @@ "." ] }, + "_LEGACY_ERROR_TEMP_0065" : { + "message" : [ + "Support for GENERATED ALWAYS AS column is not allowed." + ] + }, "_LEGACY_ERROR_TEMP_1000" : { "message" : [ "LEGACY store assignment policy is disallowed in Spark data source V2. Please set the configuration to other values." diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index d10a1f3c20dd1..7636864329af6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -1040,7 +1040,7 @@ colDefinitionOption generationExpression : GENERATED ALWAYS AS LEFT_PAREN expression RIGHT_PAREN - ; + ; complexColTypeList : complexColType (COMMA complexColType)* diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c02a2239f9cf6..f363f96d467d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2980,6 +2980,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit // Check that no duplicates exist among any CREATE TABLE column options specified. var nullable = true var defaultExpression: Option[DefaultExpressionContext] = None + var generationExpression: Option[GenerationExpressionContext] = None var commentSpec: Option[CommentSpecContext] = None ctx.colDefinitionOption().asScala.foreach { option => if (option.NULL != null) { @@ -2996,6 +2997,13 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } defaultExpression = Some(expr) } + Option(option.generationExpression()).foreach { expr => + if (generationExpression.isDefined) { + throw QueryParsingErrors.duplicateCreateTableColumnOption( + option, colName.getText, "GENERATED ALWAYS AS") + } + generationExpression = Some(expr) + } Option(option.commentSpec()).foreach { spec => if (commentSpec.isDefined) { throw QueryParsingErrors.duplicateCreateTableColumnOption( @@ -3022,7 +3030,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } // Add the 'GENERATED ALWAYS AS expression' clause in the column definition, if any, to the // column metadata. - Option(ctx.generationExpression()).map(visitGenerationExpression).foreach { field => + generationExpression.map(visitGenerationExpression).foreach { field => if (conf.getConf(SQLConf.ENABLE_GENERATED_COLUMNS)) { builder.putString(GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY, field) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 6b730de84e6d7..168f7887d0ad5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -652,6 +652,6 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def generatedColumnNotEnabledError(ctx: ParserRuleContext): Throwable = { - new ParseException("Support for GENERATED ALWAYS AS column is not allowed", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0065", ctx) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 3d4c4f97dbc52..15cb2565bb883 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2735,9 +2735,16 @@ class DDLParserSuite extends AnalysisTest { Map.empty[String, String], None, None, None, false), false)) // Make sure that the parser returns an exception when the feature is disabled. withSQLConf(SQLConf.ENABLE_GENERATED_COLUMNS.key -> "false") { - intercept( - "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet", - "Support for GENERATED ALWAYS AS column is not allowed") + val sql = "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet" + val fragment = "b INT NOT NULL GENERATED ALWAYS AS (a+1)" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0065", + parameters = Map.empty, + context = ExpectedContext( + fragment = fragment, + start = 27, + stop = 66)) } } } From 4eab27dd75907808da233475036c29509660cd53 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Tue, 29 Nov 2022 20:14:13 -0800 Subject: [PATCH 07/26] update some docs --- core/src/main/resources/error/error-classes.json | 2 +- .../spark/sql/connector/catalog/TableProvider.java | 6 +++--- .../apache/spark/sql/errors/QueryCompilationErrors.scala | 9 +++++---- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 5 +++-- .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 4 ++-- .../apache/spark/sql/connector/DataSourceV2Suite.scala | 7 ++++--- 6 files changed, 18 insertions(+), 15 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 454bdb88fc20a..9eaf7b5f7cf98 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1805,7 +1805,7 @@ }, "_LEGACY_ERROR_TEMP_0065" : { "message" : [ - "Support for GENERATED ALWAYS AS column is not allowed." + "Creating generated columns with GENERATED ALWAYS AS is not supported." ] }, "_LEGACY_ERROR_TEMP_1000" : { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index 87b8c1fd40e91..2a7f576994cbc 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -96,13 +96,13 @@ default boolean supportsExternalMetadata() { /** * Returns true if the source supports defining generated columns upon table creation in SQL. - * When false any create table statements with a generated column defined in the table schema will - * throw an exception during analysis. + * When false: any create/replace table statements with a generated column defined in the table + * schema will throw an exception during analysis. * * A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} * The generation expression is stored in the column metadata with key "generationExpression". * - * Override this method to allow defining generated columns in create table statements. + * Override this method to allow defining generated columns in create/replace table statements. */ default boolean supportsGeneratedColumnsOnCreation() { return false; 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 b3e69876f40c3..2ad428a8c451d 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 @@ -3404,12 +3404,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } } - def generatedColumnsNotAllowedInDataSource(provider: String): Throwable = { + def generatedColumnsOnCreationUnsupportedForDataSource(provider: String): Throwable = { new AnalysisException( s""" - |Failed to execute command because GENERATED ALWAYS AS expressions are not supported for - |target data source with table provider: \"$provider\" - |""".stripMargin + |Failed to execute command because target data source with table provider: \"$provider\" + |does not support creating generated columns with GENERATED ALWAYS AS expressions in + |create/replace table statements. + f|""".stripMargin ) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f607cb0b0609d..f06b7abfa06c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3049,8 +3049,9 @@ object SQLConf { val ENABLE_GENERATED_COLUMNS = buildConf("spark.sql.generatedColumn.enabled") .internal() - .doc("When true, allow CREATE TABLE statements to specify a generation expression for" + - "specific columns.") + .doc("When true, allow CREATE TABLE and REPLACE TABLE statements to define generated " + + "columns by specifying a generation expression for specific columns with " + + "`GENERATED ALWAYS AS expr`") .version("3.4.0") .booleanConf .createWithDefault(true) 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 8b89a0d52eb38..2ecbe887cfdc2 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 @@ -174,7 +174,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && !supportsGeneratedColumnsOnCreation(provider)) { - throw QueryCompilationErrors.generatedColumnsNotAllowedInDataSource(provider) + throw QueryCompilationErrors.generatedColumnsOnCreationUnsupportedForDataSource(provider) } if (!isV2Provider(provider)) { constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, @@ -211,7 +211,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && !supportsGeneratedColumnsOnCreation(provider)) { - throw QueryCompilationErrors.generatedColumnsNotAllowedInDataSource(provider) + throw QueryCompilationErrors.generatedColumnsOnCreationUnsupportedForDataSource(provider) } if (!isV2Provider(provider)) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 4c6a9dfda4e3b..0dad7817d1c5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -615,9 +615,10 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS val e = intercept[AnalysisException] { spark.sql(s"$statement $tableDefinition USING $doesNotSupportGeneratedColumn") } - assert(e.getMessage.contains("GENERATED ALWAYS AS expressions are not supported for\n" + - "target data source with table provider: " + - "\"org.apache.spark.sql.connector.SimpleWritableDataSource\"")) + assert(e.getMessage.contains("target data source with table provider: " + + "\"org.apache.spark.sql.connector.SimpleWritableDataSource\"\n" + + "does not support creating generated columns with GENERATED ALWAYS AS expressions in\n" + + "create/replace table statements")) } } } From 8c80d231cb01f3dec076f2696e82e9fdd87c0d84 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Tue, 29 Nov 2022 22:33:42 -0800 Subject: [PATCH 08/26] fix errors --- core/src/main/resources/error/error-classes.json | 15 ++++++++++----- .../spark/sql/errors/QueryCompilationErrors.scala | 7 ++----- .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../sql/catalyst/parser/DDLParserSuite.scala | 2 +- .../spark/sql/connector/DataSourceV2Suite.scala | 7 ++++--- 5 files changed, 18 insertions(+), 15 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 9eaf7b5f7cf98..9448bbf6b4f7c 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1266,6 +1266,11 @@ "DISTRIBUTE BY clause." ] }, + "GENERATED_COLUMN_ON_TABLE_CREATION": { + "message" : [ + "Target data source with table provider: \"\" does not support creating generated columns with GENERATED ALWAYS AS expressions in create/replace table statements." + ] + }, "INSERT_PARTITION_SPEC_IF_NOT_EXISTS" : { "message" : [ "INSERT INTO IF NOT EXISTS in the PARTITION spec." @@ -1331,6 +1336,11 @@ "Repeated PIVOT operation." ] }, + "SET_COLUMN_GENERATION_EXPRESSION" : { + "message" : [ + "Specifying a generation expression for a column with GENERATED ALWAYS AS expr is disabled by configuration spark.sql.generatedColumn.enabled=false" + ] + }, "SET_NAMESPACE_PROPERTY" : { "message" : [ " is a reserved namespace property, ." @@ -1803,11 +1813,6 @@ "." ] }, - "_LEGACY_ERROR_TEMP_0065" : { - "message" : [ - "Creating generated columns with GENERATED ALWAYS AS is not supported." - ] - }, "_LEGACY_ERROR_TEMP_1000" : { "message" : [ "LEGACY store assignment policy is disallowed in Spark data source V2. Please set the configuration to other values." 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 2ad428a8c451d..2f9f83d8aeaf6 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 @@ -3406,11 +3406,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def generatedColumnsOnCreationUnsupportedForDataSource(provider: String): Throwable = { new AnalysisException( - s""" - |Failed to execute command because target data source with table provider: \"$provider\" - |does not support creating generated columns with GENERATED ALWAYS AS expressions in - |create/replace table statements. - f|""".stripMargin + errorClass = "UNSUPPORTED_FEATURE.GENERATED_COLUMN_ON_TABLE_CREATION", + messageParameters = Map("provider" -> provider) ) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 168f7887d0ad5..811ed828ffefb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -652,6 +652,6 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def generatedColumnNotEnabledError(ctx: ParserRuleContext): Throwable = { - new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0065", ctx) + new ParseException(errorClass = "UNSUPPORTED_FEATURE.SET_COLUMN_GENERATION_EXPRESSION", ctx) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 15cb2565bb883..3be14af72b275 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2739,7 +2739,7 @@ class DDLParserSuite extends AnalysisTest { val fragment = "b INT NOT NULL GENERATED ALWAYS AS (a+1)" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0065", + errorClass = "UNSUPPORTED_FEATURE.SET_COLUMN_GENERATION_EXPRESSION", parameters = Map.empty, context = ExpectedContext( fragment = fragment, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 0dad7817d1c5a..c27454300700e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -615,10 +615,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS val e = intercept[AnalysisException] { spark.sql(s"$statement $tableDefinition USING $doesNotSupportGeneratedColumn") } - assert(e.getMessage.contains("target data source with table provider: " + - "\"org.apache.spark.sql.connector.SimpleWritableDataSource\"\n" + - "does not support creating generated columns with GENERATED ALWAYS AS expressions in\n" + + assert(e.getMessage.contains("Target data source with table provider: " + + "\"org.apache.spark.sql.connector.SimpleWritableDataSource\" " + + "does not support creating generated columns with GENERATED ALWAYS AS expressions in " + "create/replace table statements")) + assert(e.getErrorClass == "UNSUPPORTED_FEATURE.GENERATED_COLUMN_ON_TABLE_CREATION") } } } From 57e5c1a04a987690db9104ba3786f0d2fae11996 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Wed, 30 Nov 2022 11:35:07 -0800 Subject: [PATCH 09/26] fix formatting in error classes --- core/src/main/resources/error/error-classes.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 9448bbf6b4f7c..55185e5e4b53a 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1266,7 +1266,7 @@ "DISTRIBUTE BY clause." ] }, - "GENERATED_COLUMN_ON_TABLE_CREATION": { + "GENERATED_COLUMN_ON_TABLE_CREATION" : { "message" : [ "Target data source with table provider: \"\" does not support creating generated columns with GENERATED ALWAYS AS expressions in create/replace table statements." ] From f9999efbc0007b35358510abfdb66b01b668d323 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Fri, 2 Dec 2022 15:14:23 -0800 Subject: [PATCH 10/26] remove feature flag --- core/src/main/resources/error/error-classes.json | 5 ----- .../spark/sql/catalyst/parser/AstBuilder.scala | 6 +----- .../spark/sql/errors/QueryParsingErrors.scala | 4 ---- .../org/apache/spark/sql/internal/SQLConf.scala | 10 ---------- .../spark/sql/catalyst/parser/DDLParserSuite.scala | 13 ------------- 5 files changed, 1 insertion(+), 37 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 55185e5e4b53a..ae5c9f1c1a8f1 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1336,11 +1336,6 @@ "Repeated PIVOT operation." ] }, - "SET_COLUMN_GENERATION_EXPRESSION" : { - "message" : [ - "Specifying a generation expression for a column with GENERATED ALWAYS AS expr is disabled by configuration spark.sql.generatedColumn.enabled=false" - ] - }, "SET_NAMESPACE_PROPERTY" : { "message" : [ " is a reserved namespace property, ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f363f96d467d2..4662582aa1e43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3031,11 +3031,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit // Add the 'GENERATED ALWAYS AS expression' clause in the column definition, if any, to the // column metadata. generationExpression.map(visitGenerationExpression).foreach { field => - if (conf.getConf(SQLConf.ENABLE_GENERATED_COLUMNS)) { - builder.putString(GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY, field) - } else { - throw QueryParsingErrors.generatedColumnNotEnabledError(ctx) - } + builder.putString(GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY, field) } val name: String = colName.getText diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 811ed828ffefb..9624a06d80a64 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -650,8 +650,4 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { "optionName" -> optionName), ctx) } - - def generatedColumnNotEnabledError(ctx: ParserRuleContext): Throwable = { - new ParseException(errorClass = "UNSUPPORTED_FEATURE.SET_COLUMN_GENERATION_EXPRESSION", ctx) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f06b7abfa06c8..84d78f365acbc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3046,16 +3046,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - val ENABLE_GENERATED_COLUMNS = - buildConf("spark.sql.generatedColumn.enabled") - .internal() - .doc("When true, allow CREATE TABLE and REPLACE TABLE statements to define generated " + - "columns by specifying a generation expression for specific columns with " + - "`GENERATED ALWAYS AS expr`") - .version("3.4.0") - .booleanConf - .createWithDefault(true) - val SKIP_TYPE_VALIDATION_ON_ALTER_PARTITION = buildConf("spark.sql.legacy.skipTypeValidationOnAlterPartition") .internal() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 3be14af72b275..c321ad9259c10 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2733,18 +2733,5 @@ class DDLParserSuite extends AnalysisTest { ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithGeneratedColumn, Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), Map.empty[String, String], None, None, None, false), false)) - // Make sure that the parser returns an exception when the feature is disabled. - withSQLConf(SQLConf.ENABLE_GENERATED_COLUMNS.key -> "false") { - val sql = "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet" - val fragment = "b INT NOT NULL GENERATED ALWAYS AS (a+1)" - checkError( - exception = parseException(sql), - errorClass = "UNSUPPORTED_FEATURE.SET_COLUMN_GENERATION_EXPRESSION", - parameters = Map.empty, - context = ExpectedContext( - fragment = fragment, - start = 27, - stop = 66)) - } } } From cbb512a5a2b32a0861bcc7eadffed2e8edb4b470 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Fri, 2 Dec 2022 15:36:28 -0800 Subject: [PATCH 11/26] update error message --- core/src/main/resources/error/error-classes.json | 4 ++-- .../sql/errors/QueryCompilationErrors.scala | 6 +++--- .../analysis/ResolveSessionCatalog.scala | 6 ++++-- .../spark/sql/connector/DataSourceV2Suite.scala | 16 ++++++++-------- 4 files changed, 17 insertions(+), 15 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index ae5c9f1c1a8f1..c92b4a5951ba4 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1266,9 +1266,9 @@ "DISTRIBUTE BY clause." ] }, - "GENERATED_COLUMN_ON_TABLE_CREATION" : { + "GENERATED_COLUMN_UNSUPPORTED_FOR_PROVIDER" : { "message" : [ - "Target data source with table provider: \"\" does not support creating generated columns with GENERATED ALWAYS AS expressions in create/replace table statements." + "Target data source with table provider: \"\" does not support creating generated columns with GENERATED ALWAYS AS expressions in ." ] }, "INSERT_PARTITION_SPEC_IF_NOT_EXISTS" : { 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 2f9f83d8aeaf6..bc8ac5f27e305 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 @@ -3404,10 +3404,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } } - def generatedColumnsOnCreationUnsupportedForDataSource(provider: String): Throwable = { + def generatedColumnsUnsupportedForDataSource(provider: String, operation: String): Throwable = { new AnalysisException( - errorClass = "UNSUPPORTED_FEATURE.GENERATED_COLUMN_ON_TABLE_CREATION", - messageParameters = Map("provider" -> provider) + errorClass = "UNSUPPORTED_FEATURE.GENERATED_COLUMN_UNSUPPORTED_FOR_PROVIDER", + messageParameters = Map("provider" -> provider, "statement" -> operation) ) } } 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 2ecbe887cfdc2..74b289deb98a9 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 @@ -174,7 +174,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && !supportsGeneratedColumnsOnCreation(provider)) { - throw QueryCompilationErrors.generatedColumnsOnCreationUnsupportedForDataSource(provider) + throw QueryCompilationErrors.generatedColumnsUnsupportedForDataSource( + provider, "CREATE TABLE") } if (!isV2Provider(provider)) { constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, @@ -211,7 +212,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && !supportsGeneratedColumnsOnCreation(provider)) { - throw QueryCompilationErrors.generatedColumnsOnCreationUnsupportedForDataSource(provider) + throw QueryCompilationErrors.generatedColumnsUnsupportedForDataSource( + provider, "REPLACE TABLE") } if (!isV2Provider(provider)) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index c27454300700e..b311d6521dcd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -603,10 +603,10 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS classOf[SupportsGeneratedColumnsOnCreationWritableDataSource].getName val doesNotSupportGeneratedColumn = classOf[SimpleWritableDataSource].getName val tblName = "my_tab" - val tableDefinition = s"TABLE $tblName(a INT, b INT GENERATED ALWAYS AS (a+1))" - for (statement <- Seq("CREATE", "REPLACE")) { + val tableDefinition = s" $tblName(a INT, b INT GENERATED ALWAYS AS (a+1))" + for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { withTable(tblName) { - if (statement == "REPLACE") { + if (statement == "REPLACE TABLE") { spark.sql(s"CREATE TABLE $tblName(a INT) USING $supportsGeneratedColumn") } spark.sql(s"$statement $tableDefinition USING $supportsGeneratedColumn") @@ -615,11 +615,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS val e = intercept[AnalysisException] { spark.sql(s"$statement $tableDefinition USING $doesNotSupportGeneratedColumn") } - assert(e.getMessage.contains("Target data source with table provider: " + - "\"org.apache.spark.sql.connector.SimpleWritableDataSource\" " + - "does not support creating generated columns with GENERATED ALWAYS AS expressions in " + - "create/replace table statements")) - assert(e.getErrorClass == "UNSUPPORTED_FEATURE.GENERATED_COLUMN_ON_TABLE_CREATION") + assert(e.getMessage.contains( + "Target data source with table provider: " + + "\"org.apache.spark.sql.connector.SimpleWritableDataSource\" does not support " + + s"creating generated columns with GENERATED ALWAYS AS expressions in $statement")) + assert(e.getErrorClass == "UNSUPPORTED_FEATURE.GENERATED_COLUMN_UNSUPPORTED_FOR_PROVIDER") } } } From 7155d484b0c8ce35450f10bfe5590d346032eec4 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Wed, 14 Dec 2022 16:28:28 -0800 Subject: [PATCH 12/26] respond to comments --- core/src/main/resources/error/error-classes.json | 5 ----- .../spark/sql/connector/catalog/TableProvider.java | 4 ++-- .../spark/sql/errors/QueryCompilationErrors.scala | 12 +++++++++--- .../catalyst/analysis/ResolveSessionCatalog.scala | 4 ++-- .../spark/sql/connector/DataSourceV2Suite.scala | 8 +++----- 5 files changed, 16 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index c92b4a5951ba4..897287772011e 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1266,11 +1266,6 @@ "DISTRIBUTE BY clause." ] }, - "GENERATED_COLUMN_UNSUPPORTED_FOR_PROVIDER" : { - "message" : [ - "Target data source with table provider: \"\" does not support creating generated columns with GENERATED ALWAYS AS expressions in ." - ] - }, "INSERT_PARTITION_SPEC_IF_NOT_EXISTS" : { "message" : [ "INSERT INTO IF NOT EXISTS in the PARTITION spec." diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index 2a7f576994cbc..b8699fb1c9595 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -98,10 +98,10 @@ default boolean supportsExternalMetadata() { * Returns true if the source supports defining generated columns upon table creation in SQL. * When false: any create/replace table statements with a generated column defined in the table * schema will throw an exception during analysis. - * + *

* A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} * The generation expression is stored in the column metadata with key "generationExpression". - * + *

* Override this method to allow defining generated columns in create/replace table statements. */ default boolean supportsGeneratedColumnsOnCreation() { 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 bc8ac5f27e305..45cd50852c70c 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 @@ -3404,10 +3404,16 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } } - def generatedColumnsUnsupportedForDataSource(provider: String, operation: String): Throwable = { + def generatedColumnsUnsupportedForDataSource( + nameParts: Seq[String], + statement: String): Throwable = { new AnalysisException( - errorClass = "UNSUPPORTED_FEATURE.GENERATED_COLUMN_UNSUPPORTED_FOR_PROVIDER", - messageParameters = Map("provider" -> provider, "statement" -> operation) + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + messageParameters = Map( + "tableName" -> toSQLId(nameParts), + "operation" -> + s"creating generated columns with GENERATED ALWAYS AS expressions in $statement" + ) ) } } 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 74b289deb98a9..0a19102d76351 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 @@ -175,7 +175,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && !supportsGeneratedColumnsOnCreation(provider)) { throw QueryCompilationErrors.generatedColumnsUnsupportedForDataSource( - provider, "CREATE TABLE") + Seq(ident.catalog.get, ident.database.get, ident.table), "CREATE TABLE") } if (!isV2Provider(provider)) { constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, @@ -213,7 +213,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && !supportsGeneratedColumnsOnCreation(provider)) { throw QueryCompilationErrors.generatedColumnsUnsupportedForDataSource( - provider, "REPLACE TABLE") + Seq(ident.catalog.get, ident.database.get, ident.table), "REPLACE TABLE") } if (!isV2Provider(provider)) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index b311d6521dcd6..9744f4660be47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -615,11 +615,9 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS val e = intercept[AnalysisException] { spark.sql(s"$statement $tableDefinition USING $doesNotSupportGeneratedColumn") } - assert(e.getMessage.contains( - "Target data source with table provider: " + - "\"org.apache.spark.sql.connector.SimpleWritableDataSource\" does not support " + - s"creating generated columns with GENERATED ALWAYS AS expressions in $statement")) - assert(e.getErrorClass == "UNSUPPORTED_FEATURE.GENERATED_COLUMN_UNSUPPORTED_FOR_PROVIDER") + assert(e.getMessage.contains("does not support creating generated columns with " + + s"GENERATED ALWAYS AS expressions in $statement")) + assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") } } } From 6cf13af54ef0563be8cfaa80a9e2d206ea4376ea Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Mon, 9 Jan 2023 10:52:28 -0800 Subject: [PATCH 13/26] update to have TableCatalogCapabilities --- .../sql/connector/catalog/TableCatalog.java | 8 ++++ .../catalog/TableCatalogCapability.java | 38 +++++++++++++++++++ .../sql/connector/catalog/TableProvider.java | 14 ------- .../catalyst/analysis/v2ResolutionPlans.scala | 2 +- .../sql/catalyst/util/GeneratedColumn.scala | 17 +++++++++ .../sql/errors/QueryCompilationErrors.scala | 6 +-- .../catalog/InMemoryTableCatalog.scala | 5 +++ .../analysis/ResolveSessionCatalog.scala | 25 +----------- .../datasources/DataSourceStrategy.scala | 2 + .../datasources/v2/DataSourceV2Strategy.scala | 20 +++++++++- .../sql/connector/DataSourceV2SQLSuite.scala | 28 ++++++++++++++ .../sql/connector/DataSourceV2Suite.scala | 29 -------------- 12 files changed, 121 insertions(+), 73 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index b04c7e55138e1..823f82b552994 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -25,7 +25,9 @@ import org.apache.spark.sql.errors.QueryCompilationErrors; import org.apache.spark.sql.types.StructType; +import java.util.Collections; import java.util.Map; +import java.util.Set; /** * Catalog methods for working with Tables. @@ -78,6 +80,12 @@ public interface TableCatalog extends CatalogPlugin { */ String OPTION_PREFIX = "option."; + /** + * TODO + * @return + */ + default Set capabilities() {return Collections.emptySet();} + /** * List the tables in a namespace from the catalog. *

diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java new file mode 100644 index 0000000000000..960a0df79990a --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.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 org.apache.spark.annotation.Evolving; + +/** + * Capabilities that can be provided by a {@link TableCatalog} implementation. + *

+ * TODO + * Table catalogs use {@link TableCatalog#capabilities()} to return a set of capabilities. Each + * capability signals to Spark that the catalog supports a feature identified by the capability. + * For example, returning {@link #SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} allows Spark to + * accept {@code GENERATED ALWAYS AS} SQL syntax in {@code CREATE TABLE} statements. + * @since 3.4.0 + */ +@Evolving +public enum TableCatalogCapability { + /** + * TODO + */ + SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index b8699fb1c9595..6b68fbe123019 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -93,18 +93,4 @@ default Transform[] inferPartitioning(CaseInsensitiveStringMap options) { default boolean supportsExternalMetadata() { return false; } - - /** - * Returns true if the source supports defining generated columns upon table creation in SQL. - * When false: any create/replace table statements with a generated column defined in the table - * schema will throw an exception during analysis. - *

- * A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} - * The generation expression is stored in the column metadata with key "generationExpression". - *

- * Override this method to allow defining generated columns in create/replace table statements. - */ - default boolean supportsGeneratedColumnsOnCreation() { - return false; - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index deac4bd290126..321eecf42b09a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -241,7 +241,7 @@ case class ResolvedNonPersistentFunc( /** * A plan containing resolved identifier with catalog determined. */ -case class daResolvedIdentifier( +case class ResolvedIdentifier( catalog: CatalogPlugin, identifier: Identifier) extends LeafNodeWithoutStats { override def output: Seq[Attribute] = Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index ab19184683261..cbce2ff39610e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -40,4 +40,21 @@ object GeneratedColumn { def hasGeneratedColumns(schema: StructType): Boolean = { schema.exists(isGeneratedColumn) } + + /** + * Get the generation expression from a field if it exists. + */ + private def getGenerationExpressionStr(field: StructField): Option[String] = { + if (field.metadata.contains(GENERATION_EXPRESSION_METADATA_KEY)) { + Some(field.metadata.getString(GENERATION_EXPRESSION_METADATA_KEY)) + } else { + None + } + } + + def getGeneratedColumnsFieldToExprStr(schema: StructType): Map[StructField, String] = { + schema.filter(isGeneratedColumn).map { field => + field -> field.metadata.getString(GENERATION_EXPRESSION_METADATA_KEY) + }.toMap + } } 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 809ca036f862c..6c221178d6a66 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 @@ -3400,15 +3400,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } } - def generatedColumnsUnsupportedForDataSource( - nameParts: Seq[String], - statement: String): Throwable = { + def generatedColumnsUnsupported(nameParts: Seq[String]): AnalysisException = { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", messageParameters = Map( "tableName" -> toSQLId(nameParts), "operation" -> - s"creating generated columns with GENERATED ALWAYS AS expressions in $statement" + s"creating generated columns with GENERATED ALWAYS AS expressions" ) ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index af8070652da2d..c34fd3e01638f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -159,6 +159,11 @@ class BasicInMemoryTableCatalog extends TableCatalog { } class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamespaces { + + override def capabilities: java.util.Set[TableCatalogCapability] = { + Set(TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS).asJava + } + protected def allNamespaces: Seq[Seq[String]] = { (tables.keySet.asScala.map(_.namespace.toSeq) ++ namespaces.keySet.asScala).toSeq.distinct } 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 4bb30659233c1..1060c0614a9cd 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 @@ -25,9 +25,9 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, GeneratedColumn, ResolveDefaultColumns => DefaultCols} +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultCols} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, LookupCatalog, SupportsNamespaces, TableProvider, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, LookupCatalog, SupportsNamespaces, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.command._ @@ -171,12 +171,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) val (storageFormat, provider) = getStorageFormatAndProvider( c.tableSpec.provider, c.tableSpec.options, c.tableSpec.location, c.tableSpec.serde, ctas = false) - - if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && - !supportsGeneratedColumnsOnCreation(provider)) { - throw QueryCompilationErrors.generatedColumnsUnsupportedForDataSource( - Seq(ident.catalog.get, ident.database.get, ident.table), "CREATE TABLE") - } if (!isV2Provider(provider)) { constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, c.ignoreIfExists, storageFormat, provider) @@ -210,11 +204,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case c @ ReplaceTable(ResolvedV1Identifier(ident), _, _, _, _) => val provider = c.tableSpec.provider.getOrElse(conf.defaultDataSourceName) - if (GeneratedColumn.hasGeneratedColumns(c.tableSchema) && - !supportsGeneratedColumnsOnCreation(provider)) { - throw QueryCompilationErrors.generatedColumnsUnsupportedForDataSource( - Seq(ident.catalog.get, ident.database.get, ident.table), "REPLACE TABLE") - } if (!isV2Provider(provider)) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( Seq(ident.catalog.get, ident.database.get, ident.table), @@ -632,16 +621,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } } - private def supportsGeneratedColumnsOnCreation(provider: String): Boolean = { - // Return earlier since `lookupDataSourceV2` may fail to resolve provider "hive" to - // `HiveFileFormat`, when running tests in sql/core. - if (DDLUtils.isHiveTable(Some(provider))) return false - DataSource.lookupDataSource(provider, conf).newInstance() match { - case t: TableProvider => t.supportsGeneratedColumnsOnCreation() - case _ => false - } - } - private object DatabaseInSessionCatalog { def unapply(resolved: ResolvedNamespace): Option[String] = resolved match { case ResolvedNamespace(catalog, _) if !isSessionCatalog(catalog) => None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 8b985e82963e8..9f2dfccd5680e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -136,6 +136,8 @@ case class DataSourceAnalysis(analyzer: Analyzer) extends Rule[LogicalPlan] { val newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( tableDesc.schema, tableDesc.provider, "CREATE TABLE", false) + + // todo: block generated columns here as we do in DataSourceV2Strategy val newTableDesc = tableDesc.copy(schema = newSchema) CreateDataSourceTableCommand(newTableDesc, ignoreIfExists = mode == SaveMode.Ignore) 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 1e2a65d9ec221..171edc0011e83 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 @@ -31,8 +31,8 @@ import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.{toPrettySQL, ResolveDefaultColumns, V2ExpressionBuilder} -import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, Table, TableCapability, TableCatalog, TruncatableTable} +import org.apache.spark.sql.catalyst.util.{toPrettySQL, GeneratedColumn, ResolveDefaultColumns, V2ExpressionBuilder} +import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, Table, TableCapability, TableCatalog, TableCatalogCapability, TruncatableTable} import org.apache.spark.sql.connector.catalog.index.SupportsIndex import org.apache.spark.sql.connector.expressions.{FieldReference, LiteralValue} import org.apache.spark.sql.connector.expressions.filter.{And => V2And, Not => V2Not, Or => V2Or, Predicate} @@ -177,6 +177,14 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) + + if (GeneratedColumn.hasGeneratedColumns(newSchema)) { + if (!catalog.asTableCatalog.capabilities().contains( + TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { + throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) + } + } + CreateTableExec(catalog.asTableCatalog, ident, newSchema, partitioning, qualifyLocInTableSpec(tableSpec), ifNotExists) :: Nil @@ -200,6 +208,14 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) + + if (GeneratedColumn.hasGeneratedColumns(newSchema)) { + if (!catalog.asTableCatalog.capabilities().contains( + TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { + throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) + } + } + catalog match { case staging: StagingTableCatalog => AtomicReplaceTableExec(staging, ident, newSchema, parts, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 03b42a760eac0..956c53ce3bad3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1325,6 +1325,34 @@ class DataSourceV2SQLSuiteV1Filter } } + test("SPARK-41290") { + // test with SessionCatalog + + // val supportsGeneratedColumn = InMemoryTableCatalog + // val doesNotSupportGeneratedColumn = BasicInMemoryTableCatalog + val tblName = "my_tab" + val tableDefinition = s"$tblName(a INT, b INT GENERATED ALWAYS AS (a+1))" + for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { + withTable(tblName) { + if (statement == "REPLACE TABLE") { + spark.sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") + } + spark.sql(s"$statement testcat.$tableDefinition USING foo") + } + withTable(tblName) { + withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { + val e = intercept[AnalysisException] { + sql("USE dummy") + spark.sql(s"$statement dummy.$tableDefinition USING foo") + } + assert(e.getMessage.contains( + "does not support creating generated columns with GENERATED ALWAYS AS expressions")) + assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") + } + } + } + } + test("ShowCurrentNamespace: basic tests") { def testShowCurrentNamespace(expectedCatalogName: String, expectedNamespace: String): Unit = { val schema = new StructType() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 9744f4660be47..5c4be75e02c7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -596,31 +596,6 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS } } } - - test("SPARK-41290: cannot CREATE TABLE with GENERATED ALWAYS AS if table provider does not " + - "support generated columns on creation") { - val supportsGeneratedColumn = - classOf[SupportsGeneratedColumnsOnCreationWritableDataSource].getName - val doesNotSupportGeneratedColumn = classOf[SimpleWritableDataSource].getName - val tblName = "my_tab" - val tableDefinition = s" $tblName(a INT, b INT GENERATED ALWAYS AS (a+1))" - for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { - withTable(tblName) { - if (statement == "REPLACE TABLE") { - spark.sql(s"CREATE TABLE $tblName(a INT) USING $supportsGeneratedColumn") - } - spark.sql(s"$statement $tableDefinition USING $supportsGeneratedColumn") - } - withTable(tblName) { - val e = intercept[AnalysisException] { - spark.sql(s"$statement $tableDefinition USING $doesNotSupportGeneratedColumn") - } - assert(e.getMessage.contains("does not support creating generated columns with " + - s"GENERATED ALWAYS AS expressions in $statement")) - assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") - } - } - } } @@ -1131,7 +1106,3 @@ class ReportStatisticsDataSource extends SimpleWritableDataSource { } } } - -class SupportsGeneratedColumnsOnCreationWritableDataSource extends SimpleWritableDataSource { - override def supportsGeneratedColumnsOnCreation(): Boolean = true -} From a0eb80ed6bd794ab9d44173437bd33438ba1adf8 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Tue, 10 Jan 2023 23:05:05 -0700 Subject: [PATCH 14/26] finish up todos --- .../sql/connector/catalog/TableCatalog.java | 3 +-- .../catalog/TableCatalogCapability.java | 15 ++++++++--- .../sql/catalyst/util/GeneratedColumn.scala | 17 ------------- .../analysis/ResolveSessionCatalog.scala | 1 - .../datasources/DataSourceStrategy.scala | 9 +++++-- .../sql/connector/DataSourceV2SQLSuite.scala | 25 ++++++++----------- .../spark/sql/sources/InsertSuite.scala | 12 +++++++++ 7 files changed, 42 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index 823f82b552994..d95038ef89c34 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -81,8 +81,7 @@ public interface TableCatalog extends CatalogPlugin { String OPTION_PREFIX = "option."; /** - * TODO - * @return + * @return the set of capabilities for this TableCatalog */ default Set capabilities() {return Collections.emptySet();} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java index 960a0df79990a..2c2c8d9cbc062 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java @@ -22,17 +22,24 @@ /** * Capabilities that can be provided by a {@link TableCatalog} implementation. *

- * TODO - * Table catalogs use {@link TableCatalog#capabilities()} to return a set of capabilities. Each + * TableCatalogs use {@link TableCatalog#capabilities()} to return a set of capabilities. Each * capability signals to Spark that the catalog supports a feature identified by the capability. * For example, returning {@link #SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} allows Spark to - * accept {@code GENERATED ALWAYS AS} SQL syntax in {@code CREATE TABLE} statements. + * accept {@code GENERATED ALWAYS AS} expressions in {@code CREATE TABLE} statements. + * * @since 3.4.0 */ @Evolving public enum TableCatalogCapability { + /** - * TODO + * Signals that the TableCatalog supports defining generated columns upon table creation in SQL. + *

+ * Without this capability, any create/replace table statements with a generated column defined + * in the table schema will throw an exception during analysis. + *

+ * A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} + * The generation expression is stored in the column metadata with key "generationExpression". */ SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index cbce2ff39610e..ab19184683261 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -40,21 +40,4 @@ object GeneratedColumn { def hasGeneratedColumns(schema: StructType): Boolean = { schema.exists(isGeneratedColumn) } - - /** - * Get the generation expression from a field if it exists. - */ - private def getGenerationExpressionStr(field: StructField): Option[String] = { - if (field.metadata.contains(GENERATION_EXPRESSION_METADATA_KEY)) { - Some(field.metadata.getString(GENERATION_EXPRESSION_METADATA_KEY)) - } else { - None - } - } - - def getGeneratedColumnsFieldToExprStr(schema: StructType): Map[StructField, String] = { - schema.filter(isGeneratedColumn).map { field => - field -> field.metadata.getString(GENERATION_EXPRESSION_METADATA_KEY) - }.toMap - } } 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 1060c0614a9cd..7b2d5015840cb 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 @@ -203,7 +203,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // session catalog and the table provider is not v2. case c @ ReplaceTable(ResolvedV1Identifier(ident), _, _, _, _) => val provider = c.tableSpec.provider.getOrElse(conf.defaultDataSourceName) - if (!isV2Provider(provider)) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( Seq(ident.catalog.get, ident.database.get, ident.table), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 9f2dfccd5680e..74539b54117f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 -import org.apache.spark.sql.catalyst.util.{ResolveDefaultColumns, V2ExpressionBuilder} +import org.apache.spark.sql.catalyst.util.{GeneratedColumn, ResolveDefaultColumns, V2ExpressionBuilder} import org.apache.spark.sql.connector.catalog.SupportsRead import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.{Expression => V2Expression, NullOrdering, SortDirection, SortOrder => V2SortOrder, SortValue} @@ -137,7 +137,12 @@ case class DataSourceAnalysis(analyzer: Analyzer) extends Rule[LogicalPlan] { ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( tableDesc.schema, tableDesc.provider, "CREATE TABLE", false) - // todo: block generated columns here as we do in DataSourceV2Strategy + if (GeneratedColumn.hasGeneratedColumns(newSchema)) { + throw QueryCompilationErrors.generatedColumnsUnsupported( + Seq(tableDesc.identifier.catalog.get, tableDesc.identifier.database.get, + tableDesc.identifier.table)) + } + val newTableDesc = tableDesc.copy(schema = newSchema) CreateDataSourceTableCommand(newTableDesc, ignoreIfExists = mode == SaveMode.Ignore) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 956c53ce3bad3..36b708a5d6303 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1325,30 +1325,27 @@ class DataSourceV2SQLSuiteV1Filter } } - test("SPARK-41290") { - // test with SessionCatalog + test("SPARK-41290: Generated columns only allowed with TableCatalogs that " + + "SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS") { - // val supportsGeneratedColumn = InMemoryTableCatalog - // val doesNotSupportGeneratedColumn = BasicInMemoryTableCatalog val tblName = "my_tab" val tableDefinition = s"$tblName(a INT, b INT GENERATED ALWAYS AS (a+1))" for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { - withTable(tblName) { + // InMemoryTableCatalog.capabilities() = {SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} + withTable(s"testcat.$tblName") { if (statement == "REPLACE TABLE") { spark.sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") } spark.sql(s"$statement testcat.$tableDefinition USING foo") } - withTable(tblName) { - withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { - val e = intercept[AnalysisException] { - sql("USE dummy") - spark.sql(s"$statement dummy.$tableDefinition USING foo") - } - assert(e.getMessage.contains( - "does not support creating generated columns with GENERATED ALWAYS AS expressions")) - assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") + withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { + val e = intercept[AnalysisException] { + sql("USE dummy") + spark.sql(s"$statement dummy.$tableDefinition USING foo") } + assert(e.getMessage.contains( + "does not support creating generated columns with GENERATED ALWAYS AS expressions")) + assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index dd37c93871e08..07f90da42b895 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -2313,6 +2313,18 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } } + + test("SPARK-41290: No generated columns with V1") { + checkError( + exception = intercept[AnalysisException] { + sql(s"create table t(a int, b int generated always as (a + 1)) using parquet") + }, + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`", + "operation" -> + s"creating generated columns with GENERATED ALWAYS AS expressions") + ) + } } class FileExistingTestFileSystem extends RawLocalFileSystem { From ed41d31e4b5a554ca06787adf5c6bff46ae81321 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Tue, 10 Jan 2023 23:06:45 -0700 Subject: [PATCH 15/26] clarifying comment --- .../org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 36b708a5d6303..3054c02abe5d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1338,6 +1338,7 @@ class DataSourceV2SQLSuiteV1Filter } spark.sql(s"$statement testcat.$tableDefinition USING foo") } + // BasicInMemoryTableCatalog.capabilities() = {} withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { val e = intercept[AnalysisException] { sql("USE dummy") From 2905db1c0eaab403995dcb1a433a552543c450c9 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Mon, 6 Feb 2023 16:30:43 -0800 Subject: [PATCH 16/26] update to parse expression to v2 and then convert to sql + added tests --- .../main/resources/error/error-classes.json | 6 + .../spark/sql/connector/catalog/Table.java | 3 + .../sql/catalyst/parser/AstBuilder.scala | 6 +- .../sql/catalyst/util/GeneratedColumn.scala | 43 ------ .../sql/catalyst/util/GeneratedColumn.scala | 144 ++++++++++++++++++ .../datasources/v2/DataSourceV2Strategy.scala | 8 +- .../sql/connector/DataSourceV2SQLSuite.scala | 57 ++++++- 7 files changed, 217 insertions(+), 50 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index edf46a0fe09ec..b4bbdbe70f9c9 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1358,6 +1358,12 @@ } } }, + "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN" : { + "message" : [ + "Cannot create generated column with generation expression because it cannot be converted to a V2 expression." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_FEATURE" : { "message" : [ "The feature is not supported:" diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index 8f7a87404837c..a0940052d2ab5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -42,6 +42,9 @@ @Evolving public interface Table { + /** The metadata key for saving a generation expression in a generated column's metadata */ + String GENERATION_EXPRESSION_METADATA_KEY = "generationExpression"; + /** * A name to identify this table. Implementations should provide a meaningful name, like the * database and table name from catalog, or the location of files for this table. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index ef3df02253a31..5ef956f226c8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -41,9 +41,9 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, GeneratedColumn, IntervalUtils, ResolveDefaultColumns} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils, ResolveDefaultColumns} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTimestamp, stringToTimestampWithoutTimeZone} -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, Table, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.errors.QueryParsingErrors @@ -3029,7 +3029,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit // Add the 'GENERATED ALWAYS AS expression' clause in the column definition, if any, to the // column metadata. generationExpression.map(visitGenerationExpression).foreach { field => - builder.putString(GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY, field) + builder.putString(Table.GENERATION_EXPRESSION_METADATA_KEY, field) } val name: String = colName.getText diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala deleted file mode 100644 index ab19184683261..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.catalyst.util - -import org.apache.spark.sql.types.{StructField, StructType} - -/** - * This object contains utility methods and values for Generated Columns - */ -object GeneratedColumn { - - /** The metadata key for saving a generation expression in a generated column's metadata */ - val GENERATION_EXPRESSION_METADATA_KEY = "generationExpression" - - /** - * Whether the given `field` is a generated column - */ - private def isGeneratedColumn(field: StructField): Boolean = { - field.metadata.contains(GENERATION_EXPRESSION_METADATA_KEY) - } - - /** - * Whether the `schema` has one or more generated columns - */ - def hasGeneratedColumns(schema: StructType): Boolean = { - schema.exists(isGeneratedColumn) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala new file mode 100644 index 0000000000000..3c1519903f431 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -0,0 +1,144 @@ +/* + * 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.catalyst.util + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.analysis.Analyzer +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.BuiltInFunctionCatalog +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType} + +/** + * This object contains utility methods and values for Generated Columns + */ +object GeneratedColumn { + + /** + * Whether the given `field` is a generated column + */ + private def isGeneratedColumn(field: StructField): Boolean = { + field.metadata.contains(Table.GENERATION_EXPRESSION_METADATA_KEY) + } + + /** + * Returns the generation expression stored in the column metadata if it exists + */ + private def getGenerationExpression(field: StructField): Option[String] = { + if (isGeneratedColumn(field)) { + Some(field.metadata.getString(Table.GENERATION_EXPRESSION_METADATA_KEY)) + } else { + None + } + } + + /** + * Whether the `schema` has one or more generated columns + */ + def hasGeneratedColumns(schema: StructType): Boolean = { + schema.exists(isGeneratedColumn) + } + + /** + * Verify that `expressionStr` can be converted to V2 + * [[org.apache.spark.sql.connector.expressions.Expression]] and return the V2 expression + * as a SQL string. + * + * Throws an [[AnalysisException]] if the expression cannot be converted. + */ + private def convertToV2ExpressionSQL( + session: SparkSession, + expressionStr: String, + fieldName: String, + schema: StructType, + statementType: String): String = { + // Parse the expression string + val parsed: Expression = try { + session.sessionState.sqlParser.parseExpression(expressionStr) + } catch { + case ex: ParseException => + // Shouldn't be possible since we check that the expression is a valid catalyst expression + // during parsing + throw new AnalysisException( + s"Failed to execute $statementType command because the column $fieldName has " + + s"generation expression $expressionStr which fails to parse as a valid expression:" + + s"\n${ex.getMessage}") + } + // Analyze the parse result + // Generated column can't reference itself + val relation = new LocalRelation(StructType(schema.filterNot(_.name == fieldName)).toAttributes) + val plan = try { + val analyzer: Analyzer = GeneratedColumnAnalyzer + val analyzed = analyzer.execute(Project(Seq(Alias(parsed, fieldName)()), relation)) + analyzer.checkAnalysis(analyzed) + analyzed + } catch { + case ex: AnalysisException => + val columnList = schema.filterNot(_.name == fieldName).map(_.name).mkString("[", ",", "]") + throw new AnalysisException( + s"Failed to execute $statementType command because the column $fieldName has " + + s"generation expression $expressionStr which fails to resolve as a valid expression " + + s"given columns $columnList:" + + s"\n${ex.getMessage}") + } + val analyzed = plan.collectFirst { + case Project(Seq(a: Alias), _: LocalRelation) => a.child + }.get + // Try to convert to V2 Expression and then to SQL string + new V2ExpressionBuilder(analyzed).build().getOrElse { + throw new AnalysisException( + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + messageParameters = Map( + "fieldName" -> fieldName, + "expressionStr" -> expressionStr + ) + ) + }.toString // toString uses V2ExpressionSQLBuilder + } + + /** + * For any generated columns in `schema`, verify that the generation expression is a valid + * V2 [[org.apache.spark.sql.connector.expressions.Expression]] and convert the expression string + * to V2 Expression SQL. + */ + def verifyAndConvertToV2ExpressionSQL(session: SparkSession, + schema: StructType, statementType: String): StructType = { + val newFields = schema.map { field => + getGenerationExpression(field).map { expressionStr => + val updatedExpressionStr = + convertToV2ExpressionSQL(session, expressionStr, field.name, schema, statementType) + field.copy( + metadata = new MetadataBuilder().withMetadata(field.metadata) + .putString(Table.GENERATION_EXPRESSION_METADATA_KEY, updatedExpressionStr) + .build() + ) + }.getOrElse(field) + } + StructType(newFields) + } +} + +/** + * Analyzer for processing generated column expressions using built-in functions only. + */ +object GeneratedColumnAnalyzer extends Analyzer( + new CatalogManager(BuiltInFunctionCatalog, BuiltInFunctionCatalog.v1Catalog)) { +} 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 171edc0011e83..1d9829b7c2cd4 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 @@ -174,7 +174,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case CreateTable(ResolvedIdentifier(catalog, ident), schema, partitioning, tableSpec, ifNotExists) => - val newSchema: StructType = + var newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) @@ -183,6 +183,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) } + newSchema = GeneratedColumn.verifyAndConvertToV2ExpressionSQL( + session, newSchema, "CREATE TABLE") } CreateTableExec(catalog.asTableCatalog, ident, newSchema, @@ -205,7 +207,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat RefreshTableExec(r.catalog, r.identifier, recacheTable(r)) :: Nil case ReplaceTable(ResolvedIdentifier(catalog, ident), schema, parts, tableSpec, orCreate) => - val newSchema: StructType = + var newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) @@ -214,6 +216,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) } + newSchema = GeneratedColumn.verifyAndConvertToV2ExpressionSQL( + session, newSchema, "CREATE TABLE") } catalog match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 55f65b2ef0d15..10941fa86939d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1423,16 +1423,27 @@ class DataSourceV2SQLSuiteV1Filter test("SPARK-41290: Generated columns only allowed with TableCatalogs that " + "SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS") { - val tblName = "my_tab" - val tableDefinition = s"$tblName(a INT, b INT GENERATED ALWAYS AS (a+1))" + val testCatalog = catalog("testcat").asTableCatalog + val tableDefinition = + s"$tblName(eventDate DATE, eventYear INT GENERATED ALWAYS AS (year(eventDate)))" for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { // InMemoryTableCatalog.capabilities() = {SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} withTable(s"testcat.$tblName") { if (statement == "REPLACE TABLE") { spark.sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") } + // Can create table with a generated column spark.sql(s"$statement testcat.$tableDefinition USING foo") + + // Column metadata is updated to V2 SQL + val table = testCatalog.loadTable(Identifier.of(Array(), tblName)) + val eventYearMetadata = table.schema.collectFirst { + case f @ StructField("eventYear", _, _, _) => f.metadata + } + assert(eventYearMetadata.exists( + _.getString(Table.GENERATION_EXPRESSION_METADATA_KEY) == "EXTRACT(YEAR FROM eventDate)" + )) } // BasicInMemoryTableCatalog.capabilities() = {} withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { @@ -1447,6 +1458,48 @@ class DataSourceV2SQLSuiteV1Filter } } + test("SPARK-41290: Generated column expression must be valid V2 expression") { + // InMemoryTableCatalog.capabilities() = {SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} + val tblName = "my_tab" + withTable(s"testcat.$tblName") { + // Expression cannot be resolved since it doesn't exist + var e = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + + s"b DATE GENERATED ALWAYS AS (not_a_function(a))) USING foo") + } + assert(e.getMessage.contains("fails to resolve as a valid expression")) + + // Expression cannot be resolved since it's not a built-in function + spark.udf.register("timesTwo", (x: Int) => x * 2) + e = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + + s"b INT GENERATED ALWAYS AS (timesTwo(a))) USING foo") + } + assert(e.getMessage.contains("fails to resolve as a valid expression")) + + // Generated column can't reference itself + e = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + + s"b INT GENERATED ALWAYS AS (b + 1)) USING foo") + } + assert(e.getMessage.contains("fails to resolve as a valid expression")) + assert(e.getMessage.contains("given columns [a]")) + + // Invalid V2 expression + checkError( + exception = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + + s"b DATE GENERATED ALWAYS AS (current_date())) USING foo") + }, + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + parameters = Map( + "fieldName" -> "b", + "expressionStr" -> "current_date()" + ) + ) + } + } + test("ShowCurrentNamespace: basic tests") { def testShowCurrentNamespace(expectedCatalogName: String, expectedNamespace: String): Unit = { val schema = new StructType() From bb684cdb62425227f2492f7ec77a8c1e78d9094d Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 9 Feb 2023 19:55:14 -0800 Subject: [PATCH 17/26] fix error class order and update class doc --- core/src/main/resources/error/error-classes.json | 12 ++++++------ .../spark/sql/catalyst/util/GeneratedColumn.scala | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 1d95c7e9a5bd6..254caf5894a08 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1525,6 +1525,12 @@ }, "sqlState" : "0A000" }, + "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN" : { + "message" : [ + "Cannot create generated column with generation expression because it cannot be converted to a V2 expression." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_EXPR_FOR_OPERATOR" : { "message" : [ "A query operator contains one or more unsupported expressions. Consider to rewrite it to avoid window functions, aggregate functions, and generator functions in the WHERE clause.", @@ -1537,12 +1543,6 @@ ], "sqlState" : "42P20" }, - "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN" : { - "message" : [ - "Cannot create generated column with generation expression because it cannot be converted to a V2 expression." - ], - "sqlState" : "0A000" - }, "UNSUPPORTED_FEATURE" : { "message" : [ "The feature is not supported:" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 3c1519903f431..4e99062acf7a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType} /** - * This object contains utility methods and values for Generated Columns + * This object contains utility methods for Generated Columns */ object GeneratedColumn { From e9b28dfbfb585a8982185174309c74d16cde8281 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Wed, 22 Feb 2023 19:42:10 -0800 Subject: [PATCH 18/26] implement with new v2 column framework --- .../main/resources/error/error-classes.json | 11 +- .../spark/sql/connector/catalog/Column.java | 25 ++- .../spark/sql/connector/catalog/Table.java | 3 - .../sql/catalyst/parser/AstBuilder.scala | 6 +- .../sql/catalyst/util/GeneratedColumn.scala | 157 ++++++++++++++++++ .../sql/connector/catalog/CatalogV2Util.scala | 60 ++++--- .../sql/internal/connector/ColumnImpl.scala | 1 + .../sql/catalyst/util/GeneratedColumn.scala | 144 ---------------- .../datasources/v2/DataSourceV2Strategy.scala | 6 +- .../sql/connector/DataSourceV2SQLSuite.scala | 66 +++++--- 10 files changed, 274 insertions(+), 205 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index ea5e435c74761..953261e348714 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -571,6 +571,11 @@ ], "sqlState" : "42809" }, + "GENERATED_COLUMN_WITH_DEFAULT_VALUE" : { + "message" : [ + "A column cannot have both a default value and a generation expression but column has default value: () and generation expression: ()." + ] + }, "GRAPHITE_SINK_INVALID_PROTOCOL" : { "message" : [ "Invalid Graphite protocol: ." @@ -1609,9 +1614,9 @@ }, "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN" : { "message" : [ - "Cannot create generated column with generation expression because it cannot be converted to a V2 expression." - ], - "sqlState" : "0A000" + "Cannot create generated column with generation expression because :", + "" + ] }, "UNSUPPORTED_EXPR_FOR_OPERATOR" : { "message" : [ 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 d2c8f25e73904..6eb1a4f8e849f 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 @@ -42,7 +42,11 @@ static Column create(String name, DataType dataType) { } static Column create(String name, DataType dataType, boolean nullable) { - return create(name, dataType, nullable, null, null, null); + return create(name, dataType, nullable, null, null); + } + + static Column create(String name, DataType dataType, boolean nullable, String comment, String metadataInJSON) { + return new ColumnImpl(name, dataType, nullable, comment, null, null, metadataInJSON); } static Column create( @@ -52,7 +56,17 @@ static Column create( String comment, ColumnDefaultValue defaultValue, String metadataInJSON) { - return new ColumnImpl(name, dataType, nullable, comment, defaultValue, metadataInJSON); + return new ColumnImpl(name, dataType, nullable, comment, defaultValue, null, metadataInJSON); + } + + static Column create( + String name, + DataType dataType, + boolean nullable, + String comment, + String generationExpression, + String metadataInJSON) { + return new ColumnImpl(name, dataType, nullable, comment, null, generationExpression, metadataInJSON); } /** @@ -82,6 +96,13 @@ static Column create( @Nullable ColumnDefaultValue defaultValue(); + + /** + * Returns the generation expression of this table column. Null means no generation expression. + */ + @Nullable + String generationExpression(); + /** * Returns the column metadata in JSON format. */ diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index b646d476c676e..b9753a08aba96 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -42,9 +42,6 @@ @Evolving public interface Table { - /** The metadata key for saving a generation expression in a generated column's metadata */ - String GENERATION_EXPRESSION_METADATA_KEY = "generationExpression"; - /** * A name to identify this table. Implementations should provide a meaningful name, like the * database and table name from catalog, or the location of files for this table. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index be8b66fc5b6d0..1c5eac4ce17d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -40,9 +40,9 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils, ResolveDefaultColumns} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, GeneratedColumn, IntervalUtils, ResolveDefaultColumns} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTimestamp, stringToTimestampWithoutTimeZone} -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.errors.QueryParsingErrors @@ -3053,7 +3053,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit // Add the 'GENERATED ALWAYS AS expression' clause in the column definition, if any, to the // column metadata. generationExpression.map(visitGenerationExpression).foreach { field => - builder.putString(Table.GENERATION_EXPRESSION_METADATA_KEY, field) + builder.putString(GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY, field) } val name: String = colName.getText diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala new file mode 100644 index 0000000000000..43c9c1ee2de4c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -0,0 +1,157 @@ +/* + * 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.catalyst.util + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.Analyzer +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.BuiltInFunctionCatalog +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.errors.QueryCompilationErrors.toSQLId +import org.apache.spark.sql.types.{StructField, StructType} + +/** + * This object contains utility methods and values for Generated Columns + */ +object GeneratedColumn { + + /** The metadata key for saving a generation expression in a generated column's metadata */ + val GENERATION_EXPRESSION_METADATA_KEY = "generationExpression" + + /** Parser for parsing generation expression SQL strings */ + private lazy val parser = new CatalystSqlParser() + + /** + * Whether the given `field` is a generated column + */ + def isGeneratedColumn(field: StructField): Boolean = { + field.metadata.contains(GENERATION_EXPRESSION_METADATA_KEY) + } + + /** + * Returns the generation expression stored in the column metadata if it exists + */ + def getGenerationExpression(field: StructField): Option[String] = { + if (isGeneratedColumn(field)) { + Some(field.metadata.getString(GENERATION_EXPRESSION_METADATA_KEY)) + } else { + None + } + } + + /** + * Whether the `schema` has one or more generated columns + */ + def hasGeneratedColumns(schema: StructType): Boolean = { + schema.exists(isGeneratedColumn) + } + + /** + * Parse and analyze `expressionStr` and perform verification. This means: + * - The expression cannot refer to itself + * - No user-defined expressions + * + * Throws an [[AnalysisException]] if the expression cannot be converted or is an invalid + * generation expression according to the above rules. + */ + private def analyzeAndVerifyExpression( + expressionStr: String, + fieldName: String, + schema: StructType, + statementType: String): Unit = { + // Parse the expression string + val parsed: Expression = try { + parser.parseExpression(expressionStr) + } catch { + case ex: ParseException => + // Shouldn't be possible since we check that the expression is a valid catalyst expression + // during parsing + throw new AnalysisException( + s"Failed to execute $statementType command because the column $fieldName has " + + s"generation expression $expressionStr which fails to parse as a valid expression:" + + s"\n${ex.getMessage}") + } + // Analyze the parse result + // Generated column can't reference itself + val relation = new LocalRelation(StructType(schema.filterNot(_.name == fieldName)).toAttributes) + val plan = try { + val analyzer: Analyzer = GeneratedColumnAnalyzer + val analyzed = analyzer.execute(Project(Seq(Alias(parsed, fieldName)()), relation)) + analyzer.checkAnalysis(analyzed) + analyzed + } catch { + case ex: AnalysisException => + // Improve error message if possible + if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { + ex.messageParameters.get("objectName").filter(_ == toSQLId(fieldName)).foreach { _ => + // Generation expression references itself + throw new AnalysisException( + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + messageParameters = Map( + "fieldName" -> fieldName, + "expressionStr" -> expressionStr, + "reason" -> "generation expression cannot reference itself", + "errorMessage" -> ex.getMessage)) + } + } + if (ex.getErrorClass == "UNRESOLVED_ROUTINE") { + // Cannot resolve function using built-in catalog + ex.messageParameters.get("routineName").foreach { fnName => + throw new AnalysisException( + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + messageParameters = Map( + "fieldName" -> fieldName, + "expressionStr" -> expressionStr, + "reason" -> s"failed to resolve $fnName to a built-in function", + "errorMessage" -> ex.getMessage)) + } + } + throw new AnalysisException( + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + messageParameters = Map( + "fieldName" -> fieldName, + "expressionStr" -> expressionStr, + "reason" -> "the expression fails to resolve as a valid expression", + "errorMessage" -> ex.getMessage)) + } + val analyzed = plan.collectFirst { + case Project(Seq(a: Alias), _: LocalRelation) => a.child + }.get + // todo: additional verifications? + } + + /** + * For any generated columns in `schema`, parse, analyze and verify the generation expression. + */ + def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = { + schema.foreach { field => + getGenerationExpression(field).map { expressionStr => + analyzeAndVerifyExpression(expressionStr, field.name, schema, statementType) + } + } + } +} + +/** + * Analyzer for processing generated column expressions using built-in functions only. + */ +object GeneratedColumnAnalyzer extends Analyzer( + new CatalogManager(BuiltInFunctionCatalog, BuiltInFunctionCatalog.v1Catalog)) { +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 9b481356fa603..ef403d291775e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -22,9 +22,11 @@ import java.util.Collections import scala.collection.JavaConverters._ +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AsOfTimestamp, AsOfVersion, NamedRelation, NoSuchDatabaseException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchTableException, TimeTravelSpec} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{SerdeInfo, TableSpec} +import org.apache.spark.sql.catalyst.util.GeneratedColumn import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.connector.catalog.functions.UnboundFunction @@ -471,43 +473,63 @@ private[sql] object CatalogV2Util { /** * Converts a StructType to DS v2 columns, which decodes the StructField metadata to v2 column - * comment and default value. This is mainly used to generate DS v2 columns from table schema in - * DDL commands, so that Spark can pass DS v2 columns to DS v2 createTable and related APIs. + * comment and default value or generation expression. This is mainly used to generate DS v2 + * columns from table schema in DDL commands, so that Spark can pass DS v2 columns to DS v2 + * createTable and related APIs. */ def structTypeToV2Columns(schema: StructType): Array[Column] = { schema.fields.map(structFieldToV2Column) } private def structFieldToV2Column(f: StructField): Column = { - def createV2Column(defaultValue: ColumnDefaultValue, metadata: Metadata): Column = { - val metadataJSON = if (metadata == Metadata.empty) { + def metadataAsJson(metadata: Metadata): String = { + if (metadata == Metadata.empty) { null } else { metadata.json } - Column.create( - f.name, f.dataType, f.nullable, f.getComment().orNull, defaultValue, metadataJSON) } - if (f.getCurrentDefaultValue().isDefined && f.getExistenceDefaultValue().isDefined) { + def metadataWithKeysRemoved(keys: Seq[String]): Metadata = { + keys.foldLeft(new MetadataBuilder().withMetadata(f.metadata)) { + (builder, key) => builder.remove(key) + }.build() + } + + val isDefaultColumn = f.getCurrentDefaultValue().isDefined && + f.getExistenceDefaultValue().isDefined + val isGeneratedColumn = GeneratedColumn.isGeneratedColumn(f) + if (isDefaultColumn && isGeneratedColumn) { + // todo: should we throw this error earlier? (i.e. on parsing) + throw new AnalysisException( + errorClass = "GENERATED_COLUMN_WITH_DEFAULT_VALUE", + messageParameters = Map( + "colName" -> f.name, + "defaultValue" -> f.getCurrentDefaultValue().get, + "genExpr" -> GeneratedColumn.getGenerationExpression(f).get + ) + ) + } + + if (isDefaultColumn) { val e = analyze(f, EXISTS_DEFAULT_COLUMN_METADATA_KEY) assert(e.resolved && e.foldable, "The existence default value must be a simple SQL string that is resolved and foldable, " + "but got: " + f.getExistenceDefaultValue().get) val defaultValue = new ColumnDefaultValue( f.getCurrentDefaultValue().get, LiteralValue(e.eval(), f.dataType)) - val cleanedMetadata = new MetadataBuilder() - .withMetadata(f.metadata) - .remove("comment") - .remove(CURRENT_DEFAULT_COLUMN_METADATA_KEY) - .remove(EXISTS_DEFAULT_COLUMN_METADATA_KEY) - .build() - createV2Column(defaultValue, cleanedMetadata) + val cleanedMetadata = metadataWithKeysRemoved( + Seq("comment", CURRENT_DEFAULT_COLUMN_METADATA_KEY, EXISTS_DEFAULT_COLUMN_METADATA_KEY)) + Column.create(f.name, f.dataType, f.nullable, f.getComment().orNull, defaultValue, + metadataAsJson(cleanedMetadata)) + } else if (isGeneratedColumn) { + val cleanedMetadata = metadataWithKeysRemoved( + Seq("comment", GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY)) + Column.create(f.name, f.dataType, f.nullable, f.getComment().orNull, + GeneratedColumn.getGenerationExpression(f).get, metadataAsJson(cleanedMetadata)) } else { - val cleanedMetadata = new MetadataBuilder() - .withMetadata(f.metadata) - .remove("comment") - .build() - createV2Column(null, cleanedMetadata) + val cleanedMetadata = metadataWithKeysRemoved(Seq("comment")) + Column.create(f.name, f.dataType, f.nullable, f.getComment().orNull, + metadataAsJson(cleanedMetadata)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala index 5ab3f83eeae56..2a67ffc4bbef5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala @@ -27,4 +27,5 @@ case class ColumnImpl( nullable: Boolean, comment: String, defaultValue: ColumnDefaultValue, + generationExpression: String, metadataInJSON: String) extends Column diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala deleted file mode 100644 index 4e99062acf7a3..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ /dev/null @@ -1,144 +0,0 @@ -/* - * 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.catalyst.util - -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.analysis.Analyzer -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} -import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.BuiltInFunctionCatalog -import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType} - -/** - * This object contains utility methods for Generated Columns - */ -object GeneratedColumn { - - /** - * Whether the given `field` is a generated column - */ - private def isGeneratedColumn(field: StructField): Boolean = { - field.metadata.contains(Table.GENERATION_EXPRESSION_METADATA_KEY) - } - - /** - * Returns the generation expression stored in the column metadata if it exists - */ - private def getGenerationExpression(field: StructField): Option[String] = { - if (isGeneratedColumn(field)) { - Some(field.metadata.getString(Table.GENERATION_EXPRESSION_METADATA_KEY)) - } else { - None - } - } - - /** - * Whether the `schema` has one or more generated columns - */ - def hasGeneratedColumns(schema: StructType): Boolean = { - schema.exists(isGeneratedColumn) - } - - /** - * Verify that `expressionStr` can be converted to V2 - * [[org.apache.spark.sql.connector.expressions.Expression]] and return the V2 expression - * as a SQL string. - * - * Throws an [[AnalysisException]] if the expression cannot be converted. - */ - private def convertToV2ExpressionSQL( - session: SparkSession, - expressionStr: String, - fieldName: String, - schema: StructType, - statementType: String): String = { - // Parse the expression string - val parsed: Expression = try { - session.sessionState.sqlParser.parseExpression(expressionStr) - } catch { - case ex: ParseException => - // Shouldn't be possible since we check that the expression is a valid catalyst expression - // during parsing - throw new AnalysisException( - s"Failed to execute $statementType command because the column $fieldName has " + - s"generation expression $expressionStr which fails to parse as a valid expression:" + - s"\n${ex.getMessage}") - } - // Analyze the parse result - // Generated column can't reference itself - val relation = new LocalRelation(StructType(schema.filterNot(_.name == fieldName)).toAttributes) - val plan = try { - val analyzer: Analyzer = GeneratedColumnAnalyzer - val analyzed = analyzer.execute(Project(Seq(Alias(parsed, fieldName)()), relation)) - analyzer.checkAnalysis(analyzed) - analyzed - } catch { - case ex: AnalysisException => - val columnList = schema.filterNot(_.name == fieldName).map(_.name).mkString("[", ",", "]") - throw new AnalysisException( - s"Failed to execute $statementType command because the column $fieldName has " + - s"generation expression $expressionStr which fails to resolve as a valid expression " + - s"given columns $columnList:" + - s"\n${ex.getMessage}") - } - val analyzed = plan.collectFirst { - case Project(Seq(a: Alias), _: LocalRelation) => a.child - }.get - // Try to convert to V2 Expression and then to SQL string - new V2ExpressionBuilder(analyzed).build().getOrElse { - throw new AnalysisException( - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", - messageParameters = Map( - "fieldName" -> fieldName, - "expressionStr" -> expressionStr - ) - ) - }.toString // toString uses V2ExpressionSQLBuilder - } - - /** - * For any generated columns in `schema`, verify that the generation expression is a valid - * V2 [[org.apache.spark.sql.connector.expressions.Expression]] and convert the expression string - * to V2 Expression SQL. - */ - def verifyAndConvertToV2ExpressionSQL(session: SparkSession, - schema: StructType, statementType: String): StructType = { - val newFields = schema.map { field => - getGenerationExpression(field).map { expressionStr => - val updatedExpressionStr = - convertToV2ExpressionSQL(session, expressionStr, field.name, schema, statementType) - field.copy( - metadata = new MetadataBuilder().withMetadata(field.metadata) - .putString(Table.GENERATION_EXPRESSION_METADATA_KEY, updatedExpressionStr) - .build() - ) - }.getOrElse(field) - } - StructType(newFields) - } -} - -/** - * Analyzer for processing generated column expressions using built-in functions only. - */ -object GeneratedColumnAnalyzer extends Analyzer( - new CatalogManager(BuiltInFunctionCatalog, BuiltInFunctionCatalog.v1Catalog)) { -} 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 8ab29cd4dd879..4f906db008f38 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 @@ -184,8 +184,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) } - newSchema = GeneratedColumn.verifyAndConvertToV2ExpressionSQL( - session, newSchema, "CREATE TABLE") + GeneratedColumn.verifyGeneratedColumns(newSchema, "CREATE TABLE") } CreateTableExec(catalog.asTableCatalog, ident, structTypeToV2Columns(newSchema), // todo @@ -217,8 +216,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) } - newSchema = GeneratedColumn.verifyAndConvertToV2ExpressionSQL( - session, newSchema, "CREATE TABLE") + GeneratedColumn.verifyGeneratedColumns(newSchema, "CREATE TABLE") } val v2Columns = structTypeToV2Columns(newSchema) // todo diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index a823cf1d993c0..f4a47989b36e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1425,7 +1425,6 @@ class DataSourceV2SQLSuiteV1Filter test("SPARK-41290: Generated columns only allowed with TableCatalogs that " + "SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS") { val tblName = "my_tab" - val testCatalog = catalog("testcat").asTableCatalog val tableDefinition = s"$tblName(eventDate DATE, eventYear INT GENERATED ALWAYS AS (year(eventDate)))" for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { @@ -1436,15 +1435,6 @@ class DataSourceV2SQLSuiteV1Filter } // Can create table with a generated column spark.sql(s"$statement testcat.$tableDefinition USING foo") - - // Column metadata is updated to V2 SQL - val table = testCatalog.loadTable(Identifier.of(Array(), tblName)) - val eventYearMetadata = table.schema.collectFirst { - case f @ StructField("eventYear", _, _, _) => f.metadata - } - assert(eventYearMetadata.exists( - _.getString(Table.GENERATION_EXPRESSION_METADATA_KEY) == "EXTRACT(YEAR FROM eventDate)" - )) } // BasicInMemoryTableCatalog.capabilities() = {} withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { @@ -1459,7 +1449,32 @@ class DataSourceV2SQLSuiteV1Filter } } - test("SPARK-41290: Generated column expression must be valid V2 expression") { + test("SPARK-41290: Column cannot have both a generation expression and a default value") { + val tblName = "my_tab" + val tableDefinition = + s"$tblName(eventDate DATE, eventYear INT GENERATED ALWAYS AS (year(eventDate)) DEFAULT 0)" + withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> "foo") { + for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { + withTable(s"testcat.$tblName") { + if (statement == "REPLACE TABLE") { + spark.sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") + } + checkError( + exception = intercept[AnalysisException] { + spark.sql(s"$statement testcat.$tableDefinition USING foo") + }, + errorClass = "GENERATED_COLUMN_WITH_DEFAULT_VALUE", + parameters = Map( + "colName" -> "eventYear", + "defaultValue" -> "0", + "genExpr" -> "year(eventDate)") + ) + } + } + } + } + + test("SPARK-41290: Generated column expression must be valid generation expression") { // InMemoryTableCatalog.capabilities() = {SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} val tblName = "my_tab" withTable(s"testcat.$tblName") { @@ -1468,7 +1483,8 @@ class DataSourceV2SQLSuiteV1Filter spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + s"b DATE GENERATED ALWAYS AS (not_a_function(a))) USING foo") } - assert(e.getMessage.contains("fails to resolve as a valid expression")) + assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") + assert(e.getMessage.contains("failed to resolve `not_a_function` to a built-in function")) // Expression cannot be resolved since it's not a built-in function spark.udf.register("timesTwo", (x: Int) => x * 2) @@ -1476,28 +1492,24 @@ class DataSourceV2SQLSuiteV1Filter spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + s"b INT GENERATED ALWAYS AS (timesTwo(a))) USING foo") } - assert(e.getMessage.contains("fails to resolve as a valid expression")) + assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") + assert(e.getMessage.contains("failed to resolve `timesTwo` to a built-in function")) // Generated column can't reference itself e = intercept[AnalysisException] { spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + s"b INT GENERATED ALWAYS AS (b + 1)) USING foo") } - assert(e.getMessage.contains("fails to resolve as a valid expression")) - assert(e.getMessage.contains("given columns [a]")) + assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") + assert(e.getMessage.contains("generation expression cannot reference itself")) - // Invalid V2 expression - checkError( - exception = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + - s"b DATE GENERATED ALWAYS AS (current_date())) USING foo") - }, - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", - parameters = Map( - "fieldName" -> "b", - "expressionStr" -> "current_date()" - ) - ) + // Generated column can't reference non-existent column + e = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + + s"b INT GENERATED ALWAYS AS (c + 1)) USING foo") + } + assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") + assert(e.getMessage.contains("fails to resolve as a valid expression")) } } From 53c9dbd88752e29ba3ac9e0104e0ddcea83c2f56 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Wed, 22 Feb 2023 21:05:27 -0800 Subject: [PATCH 19/26] improve some docs --- .../org/apache/spark/sql/connector/catalog/Column.java | 6 +++++- .../sql/connector/catalog/TableCatalogCapability.java | 5 ++++- .../apache/spark/sql/catalyst/util/GeneratedColumn.scala | 7 +++++-- .../execution/datasources/v2/DataSourceV2Strategy.scala | 8 ++++---- 4 files changed, 18 insertions(+), 8 deletions(-) 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 6eb1a4f8e849f..f36876b3dd718 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 @@ -33,6 +33,8 @@ * {@link TableCatalog#createTable(Identifier, Column[], Transform[], Map)}, and report it in * {@link Table#columns()} by calling the static {@code create} functions of this interface to * create it. + *

+ * A column cannot have both a default value and a generation expression. */ @Evolving public interface Column { @@ -96,9 +98,11 @@ static Column create( @Nullable ColumnDefaultValue defaultValue(); - /** * Returns the generation expression of this table column. Null means no generation expression. + *

+ * The generation expression is stored as spark SQL dialect. It is up to the data source to verify + * expression compatibility and reject writes as necessary. */ @Nullable String generationExpression(); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java index 2c2c8d9cbc062..9662417aeab02 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java @@ -39,7 +39,10 @@ public enum TableCatalogCapability { * in the table schema will throw an exception during analysis. *

* A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} - * The generation expression is stored in the column metadata with key "generationExpression". + *

+ * Generation expression are included in the column definition for APIs like + * {@link TableCatalog#createTable}. + * See {@link Column#generationExpression()}. */ SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 43c9c1ee2de4c..9705b468b2b5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -32,7 +32,10 @@ import org.apache.spark.sql.types.{StructField, StructType} */ object GeneratedColumn { - /** The metadata key for saving a generation expression in a generated column's metadata */ + /** + * The metadata key for saving a generation expression in a generated column's metadata. This is + * only used internally and connectors should access generation expressions from the V2 columns. + */ val GENERATION_EXPRESSION_METADATA_KEY = "generationExpression" /** Parser for parsing generation expression SQL strings */ @@ -142,7 +145,7 @@ object GeneratedColumn { */ def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = { schema.foreach { field => - getGenerationExpression(field).map { expressionStr => + getGenerationExpression(field).foreach { expressionStr => analyzeAndVerifyExpression(expressionStr, field.name, schema, statementType) } } 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 4f906db008f38..a24913c919587 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 @@ -175,7 +175,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case CreateTable(ResolvedIdentifier(catalog, ident), schema, partitioning, tableSpec, ifNotExists) => - var newSchema: StructType = + val newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) @@ -187,7 +187,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat GeneratedColumn.verifyGeneratedColumns(newSchema, "CREATE TABLE") } - CreateTableExec(catalog.asTableCatalog, ident, structTypeToV2Columns(newSchema), // todo + CreateTableExec(catalog.asTableCatalog, ident, structTypeToV2Columns(newSchema), partitioning, qualifyLocInTableSpec(tableSpec), ifNotExists) :: Nil case CreateTableAsSelect(ResolvedIdentifier(catalog, ident), parts, query, tableSpec, @@ -207,7 +207,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat RefreshTableExec(r.catalog, r.identifier, recacheTable(r)) :: Nil case ReplaceTable(ResolvedIdentifier(catalog, ident), schema, parts, tableSpec, orCreate) => - var newSchema: StructType = + val newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) @@ -219,7 +219,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat GeneratedColumn.verifyGeneratedColumns(newSchema, "CREATE TABLE") } - val v2Columns = structTypeToV2Columns(newSchema) // todo + val v2Columns = structTypeToV2Columns(newSchema) catalog match { case staging: StagingTableCatalog => From 301db0d62d14c8da3cda052e4b033ba772d017c3 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 23 Feb 2023 15:30:13 -0800 Subject: [PATCH 20/26] respond to some comments --- .../spark/sql/connector/catalog/Column.java | 12 +++--- .../sql/catalyst/util/GeneratedColumn.scala | 41 +++++++++++++++---- .../sql/errors/QueryCompilationErrors.scala | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 22 +++------- .../sql/connector/DataSourceV2SQLSuite.scala | 2 +- .../sql/execution/command/DDLSuite.scala | 11 +++++ .../spark/sql/sources/InsertSuite.scala | 12 ------ 7 files changed, 56 insertions(+), 46 deletions(-) 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 f36876b3dd718..0e6e5c7741aa8 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 @@ -62,12 +62,12 @@ static Column create( } static Column create( - String name, - DataType dataType, - boolean nullable, - String comment, - String generationExpression, - String metadataInJSON) { + String name, + DataType dataType, + boolean nullable, + String comment, + String generationExpression, + String metadataInJSON) { return new ColumnImpl(name, dataType, nullable, comment, null, generationExpression, metadataInJSON); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 9705b468b2b5c..7a38db155ae2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -17,14 +17,15 @@ package org.apache.spark.sql.catalyst.util +import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.BuiltInFunctionCatalog -import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.errors.QueryCompilationErrors.toSQLId +import org.apache.spark.sql.connector.catalog.{CatalogManager, TableCatalog, TableCatalogCapability} +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.{StructField, StructType} /** @@ -75,10 +76,10 @@ object GeneratedColumn { * generation expression according to the above rules. */ private def analyzeAndVerifyExpression( - expressionStr: String, - fieldName: String, - schema: StructType, - statementType: String): Unit = { + expressionStr: String, + fieldName: String, + schema: StructType, + statementType: String): Unit = { // Parse the expression string val parsed: Expression = try { parser.parseExpression(expressionStr) @@ -86,7 +87,7 @@ object GeneratedColumn { case ex: ParseException => // Shouldn't be possible since we check that the expression is a valid catalyst expression // during parsing - throw new AnalysisException( + throw SparkException.internalError( s"Failed to execute $statementType command because the column $fieldName has " + s"generation expression $expressionStr which fails to parse as a valid expression:" + s"\n${ex.getMessage}") @@ -103,7 +104,9 @@ object GeneratedColumn { case ex: AnalysisException => // Improve error message if possible if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { - ex.messageParameters.get("objectName").filter(_ == toSQLId(fieldName)).foreach { _ => + ex.messageParameters.get("objectName") + .filter(_ == QueryCompilationErrors.toSQLId(fieldName)) + .foreach { _ => // Generation expression references itself throw new AnalysisException( errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", @@ -137,19 +140,39 @@ object GeneratedColumn { val analyzed = plan.collectFirst { case Project(Seq(a: Alias), _: LocalRelation) => a.child }.get + // todo: additional verifications? } /** * For any generated columns in `schema`, parse, analyze and verify the generation expression. */ - def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = { + private def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = { schema.foreach { field => getGenerationExpression(field).foreach { expressionStr => analyzeAndVerifyExpression(expressionStr, field.name, schema, statementType) } } } + + /** + * If `schema` contains any generated columns: + * 1) Check whether the table catalog supports generated columns otherwise throw an error. + * 2) Parse, analyze and verify the generation expressions for any generated columns. + */ + def validateGeneratedColumns( + schema: StructType, + catalog: TableCatalog, + ident: Seq[String], + statementType: String): Unit = { + if (hasGeneratedColumns(schema)) { + if (!catalog.capabilities().contains( + TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { + throw QueryCompilationErrors.generatedColumnsUnsupported(ident) + } + GeneratedColumn.verifyGeneratedColumns(schema, statementType) + } + } } /** 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 3f649f20bb347..aac067813a7de 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 @@ -3411,7 +3411,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { messageParameters = Map( "tableName" -> toSQLId(nameParts), "operation" -> - s"creating generated columns with GENERATED ALWAYS AS expressions" + s"generated columns" ) ) } 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 a24913c919587..0b0e0e34c8c64 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 @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{toPrettySQL, GeneratedColumn, ResolveDefaultColumns, V2ExpressionBuilder} -import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, Table, TableCapability, TableCatalog, TableCatalogCapability, TruncatableTable} +import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, Table, TableCapability, TableCatalog, TruncatableTable} import org.apache.spark.sql.connector.catalog.CatalogV2Util.structTypeToV2Columns import org.apache.spark.sql.connector.catalog.index.SupportsIndex import org.apache.spark.sql.connector.expressions.{FieldReference, LiteralValue} @@ -178,14 +178,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) - - if (GeneratedColumn.hasGeneratedColumns(newSchema)) { - if (!catalog.asTableCatalog.capabilities().contains( - TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { - throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) - } - GeneratedColumn.verifyGeneratedColumns(newSchema, "CREATE TABLE") - } + GeneratedColumn.validateGeneratedColumns( + newSchema, catalog.asTableCatalog, ident.asMultipartIdentifier, "CREATE TABLE") CreateTableExec(catalog.asTableCatalog, ident, structTypeToV2Columns(newSchema), partitioning, qualifyLocInTableSpec(tableSpec), ifNotExists) :: Nil @@ -210,14 +204,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val newSchema: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( schema, tableSpec.provider, "CREATE TABLE", false) - - if (GeneratedColumn.hasGeneratedColumns(newSchema)) { - if (!catalog.asTableCatalog.capabilities().contains( - TableCatalogCapability.SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS)) { - throw QueryCompilationErrors.generatedColumnsUnsupported(ident.asMultipartIdentifier) - } - GeneratedColumn.verifyGeneratedColumns(newSchema, "CREATE TABLE") - } + GeneratedColumn.validateGeneratedColumns( + newSchema, catalog.asTableCatalog, ident.asMultipartIdentifier, "CREATE TABLE") val v2Columns = structTypeToV2Columns(newSchema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index f4a47989b36e8..d6973f1c157f3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1443,7 +1443,7 @@ class DataSourceV2SQLSuiteV1Filter spark.sql(s"$statement dummy.$tableDefinition USING foo") } assert(e.getMessage.contains( - "does not support creating generated columns with GENERATED ALWAYS AS expressions")) + "does not support generated columns")) assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") } } 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 3d88d4f7ab9e3..4f3f993d7de2d 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 @@ -2181,6 +2181,17 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { assert(spark.sessionState.catalog.isRegisteredFunction(rand)) } } + + test("SPARK-41290: No generated columns with V1") { + checkError( + exception = intercept[AnalysisException] { + sql(s"create table t(a int, b int generated always as (a + 1)) using parquet") + }, + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`", + "operation" -> "generated columns") + ) + } } object FakeLocalFsFileSystem { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 3c00f3fe077a0..cc1d4ab3fcdf5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -2316,18 +2316,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } - test("SPARK-41290: No generated columns with V1") { - checkError( - exception = intercept[AnalysisException] { - sql(s"create table t(a int, b int generated always as (a + 1)) using parquet") - }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", - parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`", - "operation" -> - s"creating generated columns with GENERATED ALWAYS AS expressions") - ) - } - test("SPARK-42286: Insert into a table select from case when with cast, positive test") { withTable("t1", "t2") { sql("create table t1 (x int) using parquet") From c8dce5fe6b0b11e161fed2779d3cd5e60cccba58 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 23 Feb 2023 18:31:02 -0800 Subject: [PATCH 21/26] respond to comments --- .../main/resources/error/error-classes.json | 3 +- .../sql/catalyst/util/GeneratedColumn.scala | 62 ++++---- .../sql/errors/QueryCompilationErrors.scala | 3 +- .../sql/connector/DataSourceV2SQLSuite.scala | 147 +++++++++++++----- 4 files changed, 140 insertions(+), 75 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 953261e348714..3e75dd41b21a2 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1614,8 +1614,7 @@ }, "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN" : { "message" : [ - "Cannot create generated column with generation expression because :", - "" + "Cannot create generated column with generation expression because ." ] }, "UNSUPPORTED_EXPR_FOR_OPERATOR" : { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 7a38db155ae2e..49baad63b8315 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -20,13 +20,14 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.Analyzer -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Expression} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.BuiltInFunctionCatalog import org.apache.spark.sql.connector.catalog.{CatalogManager, TableCatalog, TableCatalogCapability} import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, StructField, StructType} /** * This object contains utility methods and values for Generated Columns @@ -71,6 +72,8 @@ object GeneratedColumn { * Parse and analyze `expressionStr` and perform verification. This means: * - The expression cannot refer to itself * - No user-defined expressions + * - The expression must be deterministic + * - The expression data type can be safely up-cast to the destination column data type * * Throws an [[AnalysisException]] if the expression cannot be converted or is an invalid * generation expression according to the above rules. @@ -78,8 +81,18 @@ object GeneratedColumn { private def analyzeAndVerifyExpression( expressionStr: String, fieldName: String, + dataType: DataType, schema: StructType, statementType: String): Unit = { + def unsupportedExpressionError(reason: String): AnalysisException = { + new AnalysisException( + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + messageParameters = Map( + "fieldName" -> fieldName, + "expressionStr" -> expressionStr, + "reason" -> reason)) + } + // Parse the expression string val parsed: Expression = try { parser.parseExpression(expressionStr) @@ -104,44 +117,33 @@ object GeneratedColumn { case ex: AnalysisException => // Improve error message if possible if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { - ex.messageParameters.get("objectName") - .filter(_ == QueryCompilationErrors.toSQLId(fieldName)) - .foreach { _ => - // Generation expression references itself - throw new AnalysisException( - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", - messageParameters = Map( - "fieldName" -> fieldName, - "expressionStr" -> expressionStr, - "reason" -> "generation expression cannot reference itself", - "errorMessage" -> ex.getMessage)) + ex.messageParameters.get("objectName").foreach { unresolvedCol => + // Check whether the unresolved column is this column (w.r.t. case-sensitivity) + if (SQLConf.get.resolver(unresolvedCol, QueryCompilationErrors.toSQLId(fieldName))) { + // Generation expression references itself + throw unsupportedExpressionError("generation expression cannot reference itself") + } } } if (ex.getErrorClass == "UNRESOLVED_ROUTINE") { // Cannot resolve function using built-in catalog ex.messageParameters.get("routineName").foreach { fnName => - throw new AnalysisException( - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", - messageParameters = Map( - "fieldName" -> fieldName, - "expressionStr" -> expressionStr, - "reason" -> s"failed to resolve $fnName to a built-in function", - "errorMessage" -> ex.getMessage)) + throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function") } } - throw new AnalysisException( - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", - messageParameters = Map( - "fieldName" -> fieldName, - "expressionStr" -> expressionStr, - "reason" -> "the expression fails to resolve as a valid expression", - "errorMessage" -> ex.getMessage)) + throw ex } val analyzed = plan.collectFirst { case Project(Seq(a: Alias), _: LocalRelation) => a.child }.get - - // todo: additional verifications? + if (!analyzed.deterministic) { + throw unsupportedExpressionError("the expression is not deterministic") + } + if (!Cast.canUpCast(analyzed.dataType, dataType)) { + throw unsupportedExpressionError( + s"the expression data type ${analyzed.dataType.simpleString} " + + s"is incompatible with column data type ${dataType.simpleString}") + } } /** @@ -150,7 +152,7 @@ object GeneratedColumn { private def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = { schema.foreach { field => getGenerationExpression(field).foreach { expressionStr => - analyzeAndVerifyExpression(expressionStr, field.name, schema, statementType) + analyzeAndVerifyExpression(expressionStr, field.name, field.dataType, schema, statementType) } } } 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 aac067813a7de..1c257966aaf4b 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 @@ -3410,8 +3410,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", messageParameters = Map( "tableName" -> toSQLId(nameParts), - "operation" -> - s"generated columns" + "operation" -> "generated columns" ) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index d6973f1c157f3..d15a0de655351 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1431,20 +1431,32 @@ class DataSourceV2SQLSuiteV1Filter // InMemoryTableCatalog.capabilities() = {SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} withTable(s"testcat.$tblName") { if (statement == "REPLACE TABLE") { - spark.sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") + sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") } // Can create table with a generated column - spark.sql(s"$statement testcat.$tableDefinition USING foo") + sql(s"$statement testcat.$tableDefinition USING foo") + assert(catalog("testcat").asTableCatalog.tableExists(Identifier.of(Array(), tblName))) } // BasicInMemoryTableCatalog.capabilities() = {} withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { - val e = intercept[AnalysisException] { - sql("USE dummy") - spark.sql(s"$statement dummy.$tableDefinition USING foo") - } - assert(e.getMessage.contains( - "does not support generated columns")) - assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") + checkError( + exception = intercept[AnalysisException] { + sql("USE dummy") + sql(s"$statement dummy.$tableDefinition USING foo") + }, + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> "`my_tab`", + "operation" -> "generated columns" + ) + ) +// val e = intercept[AnalysisException] { +// sql("USE dummy") +// sql(s"$statement dummy.$tableDefinition USING foo") +// } +// assert(e.getMessage.contains( +// "does not support generated columns")) +// assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") } } } @@ -1457,11 +1469,11 @@ class DataSourceV2SQLSuiteV1Filter for (statement <- Seq("CREATE TABLE", "REPLACE TABLE")) { withTable(s"testcat.$tblName") { if (statement == "REPLACE TABLE") { - spark.sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") + sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo") } checkError( exception = intercept[AnalysisException] { - spark.sql(s"$statement testcat.$tableDefinition USING foo") + sql(s"$statement testcat.$tableDefinition USING foo") }, errorClass = "GENERATED_COLUMN_WITH_DEFAULT_VALUE", parameters = Map( @@ -1475,41 +1487,94 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-41290: Generated column expression must be valid generation expression") { - // InMemoryTableCatalog.capabilities() = {SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS} val tblName = "my_tab" - withTable(s"testcat.$tblName") { - // Expression cannot be resolved since it doesn't exist - var e = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + - s"b DATE GENERATED ALWAYS AS (not_a_function(a))) USING foo") - } - assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") - assert(e.getMessage.contains("failed to resolve `not_a_function` to a built-in function")) - - // Expression cannot be resolved since it's not a built-in function - spark.udf.register("timesTwo", (x: Int) => x * 2) - e = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + - s"b INT GENERATED ALWAYS AS (timesTwo(a))) USING foo") + def checkUnsupportedGenerationExpression( + expr: String, + expectedReason: String, + genColType: String = "INT"): Unit = { + val tableDef = + s"CREATE TABLE testcat.$tblName(a INT, b $genColType GENERATED ALWAYS AS ($expr)) USING foo" + withTable(s"testcat.$tblName") { + checkError( + exception = intercept[AnalysisException] { + sql(tableDef) + }, + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + parameters = Map( + "fieldName" -> "b", + "expressionStr" -> expr, + "reason" -> expectedReason) + ) } - assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") - assert(e.getMessage.contains("failed to resolve `timesTwo` to a built-in function")) + } - // Generated column can't reference itself - e = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + - s"b INT GENERATED ALWAYS AS (b + 1)) USING foo") - } - assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") - assert(e.getMessage.contains("generation expression cannot reference itself")) + // Expression cannot be resolved since it doesn't exist + checkUnsupportedGenerationExpression( + "not_a_function(a)", + "failed to resolve `not_a_function` to a built-in function" + ) - // Generated column can't reference non-existent column - e = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE testcat.$tblName(a INT, " + - s"b INT GENERATED ALWAYS AS (c + 1)) USING foo") + // Expression cannot be resolved since it's not a built-in function + spark.udf.register("timesTwo", (x: Int) => x * 2) + checkUnsupportedGenerationExpression( + "timesTwo(a)", + "failed to resolve `timesTwo` to a built-in function" + ) + + // Generated column can't reference itself + checkUnsupportedGenerationExpression( + "b + 1", + "generation expression cannot reference itself" + ) + // Obeys case sensitivity when intercepting the error message + // Intercepts when case-insensitive + checkUnsupportedGenerationExpression( + "B + 1", + "generation expression cannot reference itself" + ) + // Doesn't intercept when case-sensitive + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTable(s"testcat.$tblName") { + checkError( + exception = intercept[AnalysisException] { + sql(s"CREATE TABLE testcat.$tblName(a INT, " + + s"b INT GENERATED ALWAYS AS (B + 1)) USING foo") + }, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`B`", "proposal" -> "`a`"), + context = ExpectedContext(fragment = "B", start = 0, stop = 0) + ) } - assert(e.getErrorClass == "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN") - assert(e.getMessage.contains("fails to resolve as a valid expression")) + } + + // Generated column can't reference non-existent column + withTable(s"testcat.$tblName") { + checkError( + exception = intercept[AnalysisException] { + sql(s"CREATE TABLE testcat.$tblName(a INT, b INT GENERATED ALWAYS AS (c + 1)) USING foo") + }, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`c`", "proposal" -> "`a`"), + context = ExpectedContext(fragment = "c", start = 0, stop = 0) + ) + } + + // Expression must be deterministic + checkUnsupportedGenerationExpression( + "rand()", + "the expression is not deterministic" + ) + + // Data type is incompatible + checkUnsupportedGenerationExpression( + "a + 1", + "the expression data type int is incompatible with column data type boolean", + "BOOLEAN" + ) + // But we allow valid up-casts + withTable(s"testcat.$tblName") { + sql(s"CREATE TABLE testcat.$tblName(a INT, b LONG GENERATED ALWAYS AS (a + 1)) USING foo") + assert(catalog("testcat").asTableCatalog.tableExists(Identifier.of(Array(), tblName))) } } From c0166906fe115ec308338b1891535d0f37e04cf0 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 23 Feb 2023 19:57:39 -0800 Subject: [PATCH 22/26] block subquery expressions --- .../sql/catalyst/util/GeneratedColumn.scala | 6 ++++ .../datasources/v2/DataSourceV2Strategy.scala | 1 - .../sql/connector/DataSourceV2SQLSuite.scala | 32 +++++++++++++++---- 3 files changed, 31 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 49baad63b8315..589d4e39fc8c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Expression} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.BuiltInFunctionCatalog import org.apache.spark.sql.connector.catalog.{CatalogManager, TableCatalog, TableCatalogCapability} import org.apache.spark.sql.errors.QueryCompilationErrors @@ -74,6 +75,7 @@ object GeneratedColumn { * - No user-defined expressions * - The expression must be deterministic * - The expression data type can be safely up-cast to the destination column data type + * - No subquery expressions * * Throws an [[AnalysisException]] if the expression cannot be converted or is an invalid * generation expression according to the above rules. @@ -105,6 +107,10 @@ object GeneratedColumn { s"generation expression $expressionStr which fails to parse as a valid expression:" + s"\n${ex.getMessage}") } + // Don't allow subquery expressions + if (parsed.containsPattern(PLAN_EXPRESSION)) { + throw unsupportedExpressionError("subquery expressions are not allowed for generated columns") + } // Analyze the parse result // Generated column can't reference itself val relation = new LocalRelation(StructType(schema.filterNot(_.name == fieldName)).toAttributes) 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 0b0e0e34c8c64..71ffe65b42a81 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 @@ -208,7 +208,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat newSchema, catalog.asTableCatalog, ident.asMultipartIdentifier, "CREATE TABLE") val v2Columns = structTypeToV2Columns(newSchema) - catalog match { case staging: StagingTableCatalog => AtomicReplaceTableExec(staging, ident, v2Columns, parts, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index d15a0de655351..0ce5c1549da99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1450,13 +1450,6 @@ class DataSourceV2SQLSuiteV1Filter "operation" -> "generated columns" ) ) -// val e = intercept[AnalysisException] { -// sql("USE dummy") -// sql(s"$statement dummy.$tableDefinition USING foo") -// } -// assert(e.getMessage.contains( -// "does not support generated columns")) -// assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION") } } } @@ -1576,6 +1569,31 @@ class DataSourceV2SQLSuiteV1Filter sql(s"CREATE TABLE testcat.$tblName(a INT, b LONG GENERATED ALWAYS AS (a + 1)) USING foo") assert(catalog("testcat").asTableCatalog.tableExists(Identifier.of(Array(), tblName))) } + + // No subquery expressions + checkUnsupportedGenerationExpression( + "(SELECT 1)", + "subquery expressions are not allowed for generated columns" + ) + checkUnsupportedGenerationExpression( + "(SELECT (SELECT 2) + 1)", // nested + "subquery expressions are not allowed for generated columns" + ) + checkUnsupportedGenerationExpression( + "(SELECT 1) + a", // refers to another column + "subquery expressions are not allowed for generated columns" + ) + withTable("other") { + sql("create table other(x INT) using parquet") + checkUnsupportedGenerationExpression( + "(select min(x) from other)", // refers to another table + "subquery expressions are not allowed for generated columns" + ) + } + checkUnsupportedGenerationExpression( + "(select min(x) from faketable)", // refers to a non-existent table + "subquery expressions are not allowed for generated columns" + ) } test("ShowCurrentNamespace: basic tests") { From 5b0eaad3648ab1c5c68eb860709535688c713af1 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 23 Feb 2023 20:37:36 -0800 Subject: [PATCH 23/26] can't reference other generated columns --- .../sql/catalyst/util/GeneratedColumn.scala | 13 ++++- .../sql/connector/DataSourceV2SQLSuite.scala | 48 ++++++++++++++++++- 2 files changed, 57 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 589d4e39fc8c8..b88c3f3f6010c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -112,8 +112,10 @@ object GeneratedColumn { throw unsupportedExpressionError("subquery expressions are not allowed for generated columns") } // Analyze the parse result - // Generated column can't reference itself - val relation = new LocalRelation(StructType(schema.filterNot(_.name == fieldName)).toAttributes) + val allowedBaseColumns = schema + .filterNot(_.name == fieldName) // Can't reference itself + .filterNot(isGeneratedColumn) // Can't reference other generated columns + val relation = new LocalRelation(StructType(allowedBaseColumns).toAttributes) val plan = try { val analyzer: Analyzer = GeneratedColumnAnalyzer val analyzed = analyzer.execute(Project(Seq(Alias(parsed, fieldName)()), relation)) @@ -129,6 +131,13 @@ object GeneratedColumn { // Generation expression references itself throw unsupportedExpressionError("generation expression cannot reference itself") } + // Check whether the unresolved column is another generated column in the schema + schema.filter(isGeneratedColumn).foreach { col => + if (SQLConf.get.resolver(unresolvedCol, QueryCompilationErrors.toSQLId(col.name))) { + throw unsupportedExpressionError( + "generation expression cannot reference another generated column") + } + } } } if (ex.getErrorClass == "UNRESOLVED_ROUTINE") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 0ce5c1549da99..1fdaafb8add14 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1484,13 +1484,14 @@ class DataSourceV2SQLSuiteV1Filter def checkUnsupportedGenerationExpression( expr: String, expectedReason: String, - genColType: String = "INT"): Unit = { + genColType: String = "INT", + customTableDef: Option[String] = None): Unit = { val tableDef = s"CREATE TABLE testcat.$tblName(a INT, b $genColType GENERATED ALWAYS AS ($expr)) USING foo" withTable(s"testcat.$tblName") { checkError( exception = intercept[AnalysisException] { - sql(tableDef) + sql(customTableDef.getOrElse(tableDef)) }, errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", parameters = Map( @@ -1539,6 +1540,49 @@ class DataSourceV2SQLSuiteV1Filter ) } } + // Respects case sensitivity when resolving + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTable(s"testcat.$tblName") { + sql(s"CREATE TABLE testcat.$tblName(" + + s"a INT, b INT GENERATED ALWAYS AS (B + 1), B INT) USING foo") + assert(catalog("testcat").asTableCatalog.tableExists(Identifier.of(Array(), tblName))) + } + } + + // Generated column can't reference other generated columns + checkUnsupportedGenerationExpression( + "c + 1", + "generation expression cannot reference another generated column", + customTableDef = Some( + s"CREATE TABLE testcat.$tblName(a INT, " + + s"b INT GENERATED ALWAYS AS (c + 1), c INT GENERATED ALWAYS AS (a + 1)) USING foo" + ) + ) + // Is case-insensitive by default + checkUnsupportedGenerationExpression( + "C + 1", + "generation expression cannot reference another generated column", + customTableDef = Some( + s"CREATE TABLE testcat.$tblName(a INT, " + + s"b INT GENERATED ALWAYS AS (C + 1), c INT GENERATED ALWAYS AS (a + 1)) USING foo" + ) + ) + checkUnsupportedGenerationExpression( + "c + 1", + "generation expression cannot reference another generated column", + customTableDef = Some( + s"CREATE TABLE testcat.$tblName(a INT, " + + s"b INT GENERATED ALWAYS AS (c + 1), C INT GENERATED ALWAYS AS (a + 1)) USING foo" + ) + ) + // Respects case sensitivity when resolving + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTable(s"testcat.$tblName") { + sql(s"CREATE TABLE testcat.$tblName(" + + s"a INT, A INT GENERATED ALWAYS AS (a + 1), b INT GENERATED ALWAYS AS (a + 1)) USING foo") + assert(catalog("testcat").asTableCatalog.tableExists(Identifier.of(Array(), tblName))) + } + } // Generated column can't reference non-existent column withTable(s"testcat.$tblName") { From f01a0b4963adec20c7d12a4ddb3247d43f275d42 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 23 Feb 2023 21:10:46 -0800 Subject: [PATCH 24/26] respond to some comments --- .../sql/connector/catalog/TableCatalog.java | 2 +- .../catalog/TableCatalogCapability.java | 26 ++++----- .../sql/catalyst/util/GeneratedColumn.scala | 12 ++-- .../sql/connector/catalog/CatalogV2Util.scala | 1 - .../sql/catalyst/parser/DDLParserSuite.scala | 26 +++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 58 ++++++++++--------- 6 files changed, 78 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index fd259a1d073d3..eb442ad38bde5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -83,7 +83,7 @@ public interface TableCatalog extends CatalogPlugin { /** * @return the set of capabilities for this TableCatalog */ - default Set capabilities() {return Collections.emptySet();} + default Set capabilities() { return Collections.emptySet(); } /** * List the tables in a namespace from the catalog. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java index 9662417aeab02..84a2a0f76481e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java @@ -32,17 +32,17 @@ @Evolving public enum TableCatalogCapability { - /** - * Signals that the TableCatalog supports defining generated columns upon table creation in SQL. - *

- * Without this capability, any create/replace table statements with a generated column defined - * in the table schema will throw an exception during analysis. - *

- * A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} - *

- * Generation expression are included in the column definition for APIs like - * {@link TableCatalog#createTable}. - * See {@link Column#generationExpression()}. - */ - SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS + /** + * Signals that the TableCatalog supports defining generated columns upon table creation in SQL. + *

+ * Without this capability, any create/replace table statements with a generated column defined + * in the table schema will throw an exception during analysis. + *

+ * A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)} + *

+ * Generation expression are included in the column definition for APIs like + * {@link TableCatalog#createTable}. + * See {@link Column#generationExpression()}. + */ + SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index b88c3f3f6010c..a0ad958794131 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -71,7 +71,8 @@ object GeneratedColumn { /** * Parse and analyze `expressionStr` and perform verification. This means: - * - The expression cannot refer to itself + * - The expression cannot reference itself + * - The expression cannot reference other generated columns * - No user-defined expressions * - The expression must be deterministic * - The expression data type can be safely up-cast to the destination column data type @@ -127,13 +128,14 @@ object GeneratedColumn { if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { ex.messageParameters.get("objectName").foreach { unresolvedCol => // Check whether the unresolved column is this column (w.r.t. case-sensitivity) - if (SQLConf.get.resolver(unresolvedCol, QueryCompilationErrors.toSQLId(fieldName))) { + val resolver = SQLConf.get.resolver + if (resolver(unresolvedCol, QueryCompilationErrors.toSQLId(fieldName))) { // Generation expression references itself throw unsupportedExpressionError("generation expression cannot reference itself") } // Check whether the unresolved column is another generated column in the schema schema.filter(isGeneratedColumn).foreach { col => - if (SQLConf.get.resolver(unresolvedCol, QueryCompilationErrors.toSQLId(col.name))) { + if (resolver(unresolvedCol, QueryCompilationErrors.toSQLId(col.name))) { throw unsupportedExpressionError( "generation expression cannot reference another generated column") } @@ -165,7 +167,7 @@ object GeneratedColumn { * For any generated columns in `schema`, parse, analyze and verify the generation expression. */ private def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = { - schema.foreach { field => + schema.foreach { field => getGenerationExpression(field).foreach { expressionStr => analyzeAndVerifyExpression(expressionStr, field.name, field.dataType, schema, statementType) } @@ -174,7 +176,7 @@ object GeneratedColumn { /** * If `schema` contains any generated columns: - * 1) Check whether the table catalog supports generated columns otherwise throw an error. + * 1) Check whether the table catalog supports generated columns. Otherwise throw an error. * 2) Parse, analyze and verify the generation expressions for any generated columns. */ def validateGeneratedColumns( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index ef403d291775e..12a8db9236357 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -499,7 +499,6 @@ private[sql] object CatalogV2Util { f.getExistenceDefaultValue().isDefined val isGeneratedColumn = GeneratedColumn.isGeneratedColumn(f) if (isDefaultColumn && isGeneratedColumn) { - // todo: should we throw this error earlier? (i.e. on parsing) throw new AnalysisException( errorClass = "GENERATED_COLUMN_WITH_DEFAULT_VALUE", messageParameters = Map( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 728623afdf334..a1971a68bc255 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2733,5 +2733,31 @@ class DDLParserSuite extends AnalysisTest { ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithGeneratedColumn, Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), Map.empty[String, String], None, None, None, false), false)) + // Two generation expressions + checkError( + exception = parseException("CREATE TABLE my_tab(a INT, " + + "b INT GENERATED ALWAYS AS (a + 1) GENERATED ALWAYS AS (a + 2)) USING PARQUET"), + errorClass = "CREATE_TABLE_COLUMN_OPTION_DUPLICATE", + parameters = Map("columnName" -> "b", "optionName" -> "GENERATED ALWAYS AS"), + context = ExpectedContext( + fragment = "b INT GENERATED ALWAYS AS (a + 1) GENERATED ALWAYS AS (a + 2)", + start = 27, + stop = 87 + ) + ) + // Empty expression + checkError( + exception = parseException( + "CREATE TABLE my_tab(a INT, b INT GENERATED ALWAYS AS ()) USING PARQUET"), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "')'", "hint" -> "") + ) + // No parenthesis + checkError( + exception = parseException( + "CREATE TABLE my_tab(a INT, b INT GENERATED ALWAYS AS a + 1) USING PARQUET"), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'a'", "hint" -> ": missing '('") + ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 1fdaafb8add14..dc8dc75d742f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1522,17 +1522,19 @@ class DataSourceV2SQLSuiteV1Filter ) // Obeys case sensitivity when intercepting the error message // Intercepts when case-insensitive - checkUnsupportedGenerationExpression( - "B + 1", - "generation expression cannot reference itself" - ) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkUnsupportedGenerationExpression( + "B + 1", + "generation expression cannot reference itself" + ) + } // Doesn't intercept when case-sensitive - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { withTable(s"testcat.$tblName") { checkError( exception = intercept[AnalysisException] { sql(s"CREATE TABLE testcat.$tblName(a INT, " + - s"b INT GENERATED ALWAYS AS (B + 1)) USING foo") + "b INT GENERATED ALWAYS AS (B + 1)) USING foo") }, errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`B`", "proposal" -> "`a`"), @@ -1541,45 +1543,47 @@ class DataSourceV2SQLSuiteV1Filter } } // Respects case sensitivity when resolving - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { withTable(s"testcat.$tblName") { sql(s"CREATE TABLE testcat.$tblName(" + - s"a INT, b INT GENERATED ALWAYS AS (B + 1), B INT) USING foo") + "a INT, b INT GENERATED ALWAYS AS (B + 1), B INT) USING foo") assert(catalog("testcat").asTableCatalog.tableExists(Identifier.of(Array(), tblName))) } } - + // Generated column can't reference other generated columns checkUnsupportedGenerationExpression( "c + 1", "generation expression cannot reference another generated column", customTableDef = Some( s"CREATE TABLE testcat.$tblName(a INT, " + - s"b INT GENERATED ALWAYS AS (c + 1), c INT GENERATED ALWAYS AS (a + 1)) USING foo" + "b INT GENERATED ALWAYS AS (c + 1), c INT GENERATED ALWAYS AS (a + 1)) USING foo" ) ) - // Is case-insensitive by default - checkUnsupportedGenerationExpression( - "C + 1", - "generation expression cannot reference another generated column", - customTableDef = Some( - s"CREATE TABLE testcat.$tblName(a INT, " + - s"b INT GENERATED ALWAYS AS (C + 1), c INT GENERATED ALWAYS AS (a + 1)) USING foo" + // Respects case-insensitivity + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkUnsupportedGenerationExpression( + "C + 1", + "generation expression cannot reference another generated column", + customTableDef = Some( + s"CREATE TABLE testcat.$tblName(a INT, " + + "b INT GENERATED ALWAYS AS (C + 1), c INT GENERATED ALWAYS AS (a + 1)) USING foo" + ) ) - ) - checkUnsupportedGenerationExpression( - "c + 1", - "generation expression cannot reference another generated column", - customTableDef = Some( - s"CREATE TABLE testcat.$tblName(a INT, " + - s"b INT GENERATED ALWAYS AS (c + 1), C INT GENERATED ALWAYS AS (a + 1)) USING foo" + checkUnsupportedGenerationExpression( + "c + 1", + "generation expression cannot reference another generated column", + customTableDef = Some( + s"CREATE TABLE testcat.$tblName(a INT, " + + "b INT GENERATED ALWAYS AS (c + 1), C INT GENERATED ALWAYS AS (a + 1)) USING foo" + ) ) - ) + } // Respects case sensitivity when resolving - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { withTable(s"testcat.$tblName") { sql(s"CREATE TABLE testcat.$tblName(" + - s"a INT, A INT GENERATED ALWAYS AS (a + 1), b INT GENERATED ALWAYS AS (a + 1)) USING foo") + "a INT, A INT GENERATED ALWAYS AS (a + 1), b INT GENERATED ALWAYS AS (a + 1)) USING foo") assert(catalog("testcat").asTableCatalog.tableExists(Identifier.of(Array(), tblName))) } } From 09d437d3cb901ae967c01ea14e1834d0cfee8e5b Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Fri, 24 Feb 2023 12:59:22 -0800 Subject: [PATCH 25/26] respond to comments --- .../main/resources/error/error-classes.json | 2 +- .../sql/catalyst/util/GeneratedColumn.scala | 24 +++++++++---------- .../sql/connector/DataSourceV2SQLSuite.scala | 4 ++-- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 3e75dd41b21a2..408c97acaa39c 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -573,7 +573,7 @@ }, "GENERATED_COLUMN_WITH_DEFAULT_VALUE" : { "message" : [ - "A column cannot have both a default value and a generation expression but column has default value: () and generation expression: ()." + "A column cannot have both a default value and a generation expression but column has default value: () and generation expression: ()." ] }, "GRAPHITE_SINK_INVALID_PROTOCOL" : { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index a0ad958794131..6ff5df98d3ccf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -39,7 +39,7 @@ object GeneratedColumn { * The metadata key for saving a generation expression in a generated column's metadata. This is * only used internally and connectors should access generation expressions from the V2 columns. */ - val GENERATION_EXPRESSION_METADATA_KEY = "generationExpression" + val GENERATION_EXPRESSION_METADATA_KEY = "GENERATION_EXPRESSION" /** Parser for parsing generation expression SQL strings */ private lazy val parser = new CatalystSqlParser() @@ -112,7 +112,7 @@ object GeneratedColumn { if (parsed.containsPattern(PLAN_EXPRESSION)) { throw unsupportedExpressionError("subquery expressions are not allowed for generated columns") } - // Analyze the parse result + // Analyze the parsed result val allowedBaseColumns = schema .filterNot(_.name == fieldName) // Can't reference itself .filterNot(isGeneratedColumn) // Can't reference other generated columns @@ -127,18 +127,18 @@ object GeneratedColumn { // Improve error message if possible if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { ex.messageParameters.get("objectName").foreach { unresolvedCol => - // Check whether the unresolved column is this column (w.r.t. case-sensitivity) val resolver = SQLConf.get.resolver - if (resolver(unresolvedCol, QueryCompilationErrors.toSQLId(fieldName))) { - // Generation expression references itself + // Whether `col` = `unresolvedCol` taking into account case-sensitivity + def isUnresolvedCol(col: String) = + resolver(unresolvedCol, QueryCompilationErrors.toSQLId(col)) + // Check whether the unresolved column is this column + if (isUnresolvedCol(fieldName)) { throw unsupportedExpressionError("generation expression cannot reference itself") } // Check whether the unresolved column is another generated column in the schema - schema.filter(isGeneratedColumn).foreach { col => - if (resolver(unresolvedCol, QueryCompilationErrors.toSQLId(col.name))) { - throw unsupportedExpressionError( - "generation expression cannot reference another generated column") - } + if (schema.exists(col => isGeneratedColumn(col) && isUnresolvedCol(col.name))) { + throw unsupportedExpressionError( + "generation expression cannot reference another generated column") } } } @@ -154,11 +154,11 @@ object GeneratedColumn { case Project(Seq(a: Alias), _: LocalRelation) => a.child }.get if (!analyzed.deterministic) { - throw unsupportedExpressionError("the expression is not deterministic") + throw unsupportedExpressionError("generation expression is not deterministic") } if (!Cast.canUpCast(analyzed.dataType, dataType)) { throw unsupportedExpressionError( - s"the expression data type ${analyzed.dataType.simpleString} " + + s"generation expression data type ${analyzed.dataType.simpleString} " + s"is incompatible with column data type ${dataType.simpleString}") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index dc8dc75d742f7..85984f1b2a835 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1603,13 +1603,13 @@ class DataSourceV2SQLSuiteV1Filter // Expression must be deterministic checkUnsupportedGenerationExpression( "rand()", - "the expression is not deterministic" + "generation expression is not deterministic" ) // Data type is incompatible checkUnsupportedGenerationExpression( "a + 1", - "the expression data type int is incompatible with column data type boolean", + "generation expression data type int is incompatible with column data type boolean", "BOOLEAN" ) // But we allow valid up-casts From 3fb71ed8641ddb5bdf0163cc09af10f932a04a49 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Fri, 24 Feb 2023 16:27:04 -0800 Subject: [PATCH 26/26] fix github actions --- .../org/apache/spark/sql/connector/catalog/Column.java | 10 ++++++++-- .../spark/sql/catalyst/parser/DDLParserSuite.scala | 6 ++++-- 2 files changed, 12 insertions(+), 4 deletions(-) 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 0e6e5c7741aa8..b191438dbc3ee 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 @@ -47,7 +47,12 @@ static Column create(String name, DataType dataType, boolean nullable) { return create(name, dataType, nullable, null, null); } - static Column create(String name, DataType dataType, boolean nullable, String comment, String metadataInJSON) { + static Column create( + String name, + DataType dataType, + boolean nullable, + String comment, + String metadataInJSON) { return new ColumnImpl(name, dataType, nullable, comment, null, null, metadataInJSON); } @@ -68,7 +73,8 @@ static Column create( String comment, String generationExpression, String metadataInJSON) { - return new ColumnImpl(name, dataType, nullable, comment, null, generationExpression, metadataInJSON); + return new ColumnImpl(name, dataType, nullable, comment, null, + generationExpression, metadataInJSON); } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index a1971a68bc255..5196d19ffcd9b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{EqualTo, Hex, Literal} import org.apache.spark.sql.catalyst.plans.logical.{TableSpec => LogicalTableSpec, _} -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns +import org.apache.spark.sql.catalyst.util.{GeneratedColumn, ResolveDefaultColumns} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition.{after, first} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.connector.expressions.LogicalExpressions.bucket @@ -2722,7 +2722,9 @@ class DDLParserSuite extends AnalysisTest { val schemaWithGeneratedColumn = new StructType() .add("a", IntegerType, true) .add("b", IntegerType, false, - new MetadataBuilder().putString("generationExpression", "a+1").build()) + new MetadataBuilder() + .putString(GeneratedColumn.GENERATION_EXPRESSION_METADATA_KEY, "a+1") + .build()) comparePlans(parsePlan( "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), CreateTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithGeneratedColumn,