You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/11/28 18:28:28 UTC

[GitHub] [spark] allisonport-db opened a new pull request, #38823: [SPARK-41290] Support defining generated columns in create table (support GENERATED ALWAYS AS syntax)

allisonport-db opened a new pull request, #38823:
URL: https://github.com/apache/spark/pull/38823

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] RussellSpitzer commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1049247458


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   This really should be a part of the greater the catalog capabilities since that's where create table is usually going to be invoked. I'm very nervous with saying that it is up to the Datasource to decide what is valid because different engines may decide the same sql means different things and this would require that the Datasource somehow make sure non spark engines can access the same table in the same. We spent a lot of time making an public expression class for the connectors but it feels like that should probably be invoked here as well?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1092140673


##########
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"

Review Comment:
   This needs to be public as data sources need to know this special column property, so that they can provide the generate expression string to other engines. How about we put it as a static field in `org.apache.spark.sql.connector.catalog.Table`? This is similar to the special table properties defined in `TableCatalog`.
   
   We can still keep the below 2 functions here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1443038314

   > Are we going to document this new SQL syntax somewhere
   
   This is a long-standing problem. There are quite some features that only have DS v2 API but no implementation, e.g. UPDATE, MERGE, DELETE. I think we should make the testing in-memory catalog/table a builtin connector, so that we can document these SQL syntaxes with valid examples. cc @huaxingao 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116394927


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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") {

Review Comment:
   My bad, the way you were doing it before is fine



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116457448


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala:
##########
@@ -201,7 +204,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val newSchema: StructType =
         ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults(
           schema, tableSpec.provider, "CREATE TABLE", false)
+      GeneratedColumn.validateGeneratedColumns(
+        newSchema, catalog.asTableCatalog, ident.asMultipartIdentifier, "CREATE TABLE")
+
       val v2Columns = structTypeToV2Columns(newSchema)
+

Review Comment:
   nit: unnecessary change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116468829


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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.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.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, 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. 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 */
+  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
+   * - 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.
+   */
+  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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // 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").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 unsupportedExpressionError(s"failed to resolve $fnName to a built-in function")
+          }
+        }
+        throw ex
+    }
+    val analyzed = plan.collectFirst {
+      case Project(Seq(a: Alias), _: LocalRelation) => a.child
+    }.get
+    if (!analyzed.deterministic) {
+      throw unsupportedExpressionError("the expression is not deterministic")
+    }

Review Comment:
   yes adding



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1043022539


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   then it should be in `Table`? In the ideal ds v2 code path, `TableProvider` is not event used: `TableCatalog` -> `Table` -> `ScanBuilder` ...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1445917688

   thanks, merging to master/3.4!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115293400


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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)

Review Comment:
   does the column order matter? e.g. `CREATE TABLE t(c1 int, c2 int ... generated as ..., c3 int)`, can generate expression of c2 reference c3?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115290604


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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,

Review Comment:
   nit: 4 spaces indentation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115298759


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -3405,6 +3405,17 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
     }
   }
 
+  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"

Review Comment:
   this should be a short phrase. how about just `generated columns`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115300391


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala:
##########
@@ -178,6 +178,15 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val newSchema: StructType =
         ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults(
           schema, tableSpec.provider, "CREATE TABLE", false)
+
+      if (GeneratedColumn.hasGeneratedColumns(newSchema)) {

Review Comment:
   shall we move this check to a util function in `GeneratedColumn`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116440360


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -1422,6 +1422,97 @@ 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(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")
+      }
+      // 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 creating generated columns with GENERATED ALWAYS AS expressions"))
+        assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION")
+      }
+    }
+  }
+
+  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") {
+      // 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"))

Review Comment:
   Updated all of them



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116486630


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala:
##########
@@ -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)

Review Comment:
   let's remove this TODO then.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
zsxwing commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1042918249


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   We can just call this method `supportsGeneratedColumns`, so that we don't need to add a new API if we want to extend generated columns in other statements in future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115292891


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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(

Review Comment:
   if this shouldn't happen, we should throw `SparkException.internalError`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116469149


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -1422,6 +1422,162 @@ 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(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") {
+          sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo")
+        }
+        // Can create table with a generated column
+        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) {
+        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] {

Review Comment:
   accidentally included, it's identical to the `checkError` above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116490968


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala:
##########
@@ -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)

Review Comment:
   +1 for the removal.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116491490


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala:
##########
@@ -2717,4 +2717,21 @@ class DDLParserSuite extends AnalysisTest {
       context = ExpectedContext(
         fragment = "b STRING COMMENT \"abc\" NOT NULL COMMENT \"abc\"", start = 27, stop = 71))
   }
+
+  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,
+        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"),

Review Comment:
   +1 for adding more negative test cases with `checkError`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115297770


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala:
##########
@@ -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)

Review Comment:
   it's better to fail at analysis time, as SQL is not the only API, e.g. Spark Connect.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116498787


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -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"

Review Comment:
   ditto. `s"` -> `"`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116498272


##########
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

Review Comment:
   Shall we add this to the function description after line 74, too?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1098055571


##########
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"

Review Comment:
   Moved this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wangyepeng2 commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "wangyepeng2 (via GitHub)" <gi...@apache.org>.
wangyepeng2 commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1413345197

   This feature is very useful for us ,will it be released in 3.4?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
zsxwing commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1042851866


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   But whether supporting generated columns should be a data source property, rather than a catalog property?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116440888


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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 { _ =>

Review Comment:
   Updated this to account for case-sensitivity based on config "spark.sql.caseSensitive" and added a comment. Also added tests for this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116440211


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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

Review Comment:
   The analyzer handles the case sensitivity. This is just filtering out the current column from the schema, so equivalence is fine (since `fieldName = field.name`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116510940


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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
+   * - 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.
+   */
+  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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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)
+            if (SQLConf.get.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))) {
+                throw unsupportedExpressionError(
+                  "generation expression cannot reference another generated column")
+              }
+            }
+          }
+        }
+        if (ex.getErrorClass == "UNRESOLVED_ROUTINE") {
+          // Cannot resolve function using built-in catalog
+          ex.messageParameters.get("routineName").foreach { fnName =>
+            throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function")
+          }
+        }
+        throw ex
+    }
+    val analyzed = plan.collectFirst {
+      case Project(Seq(a: Alias), _: LocalRelation) => a.child
+    }.get
+    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}")
+    }
+  }
+
+  /**
+   * For any generated columns in `schema`, parse, analyze and verify the generation expression.
+   */
+  private def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = {
+   schema.foreach { field =>
+      getGenerationExpression(field).foreach { expressionStr =>
+        analyzeAndVerifyExpression(expressionStr, field.name, field.dataType, schema, statementType)
+      }
+    }
+  }
+
+  /**
+   * If `schema` contains any generated columns:
+   * 1) Check whether the table catalog supports generated columns otherwise throw an error.

Review Comment:
   Maybe, `columns otherwise ` -> `columns. Otherwise`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116510520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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
+   * - 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.
+   */
+  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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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)
+            if (SQLConf.get.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))) {
+                throw unsupportedExpressionError(
+                  "generation expression cannot reference another generated column")
+              }
+            }
+          }
+        }
+        if (ex.getErrorClass == "UNRESOLVED_ROUTINE") {
+          // Cannot resolve function using built-in catalog
+          ex.messageParameters.get("routineName").foreach { fnName =>
+            throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function")
+          }
+        }
+        throw ex
+    }
+    val analyzed = plan.collectFirst {
+      case Project(Seq(a: Alias), _: LocalRelation) => a.child
+    }.get
+    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}")
+    }
+  }
+
+  /**
+   * For any generated columns in `schema`, parse, analyze and verify the generation expression.
+   */
+  private def verifyGeneratedColumns(schema: StructType, statementType: String): Unit = {
+   schema.foreach { field =>

Review Comment:
   Indentation? We need one more space.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116493655


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalogCapability.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ * <p>
+ * 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} expressions in {@code CREATE TABLE} statements.
+ *
+ * @since 3.4.0
+ */
+@Evolving
+public enum TableCatalogCapability {
+
+    /**
+     * Signals that the TableCatalog supports defining generated columns upon table creation in SQL.
+     * <p>
+     * Without this capability, any create/replace table statements with a generated column defined
+     * in the table schema will throw an exception during analysis.
+     * <p>
+     * A generated column is defined with syntax: {@code colName colType GENERATED ALWAYS AS (expr)}
+     * <p>
+     * Generation expression are included in the column definition for APIs like
+     * {@link TableCatalog#createTable}.
+     * See {@link Column#generationExpression()}.
+     */
+    SUPPORTS_CREATE_TABLE_WITH_GENERATED_COLUMNS

Review Comment:
   We use `2-space` indentation in Java files, don't we?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1039264107


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   It's a bit weird to define this API in `TableProvider`, which doesn't even have `createTable` function. I think `TableCatalog` is a better place.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1336911874

   Does the SQL standard say anything about the restrictions of the generate expression? Can we allow `GENERATED AS rand()`? I think at least catalyst should have a rule to check the expression and make sure the data type is the same as column type.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1049104281


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   @sunchao `TableCapability` has the same issue described above as including it in `Table`
   @RussellSpitzer I think it would be up to the implementing data source to restrict what's supported?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116029029


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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

Review Comment:
   please expand this to provide more background on what generated columns are, the SQL syntax to specify them, and links to other files or places where the reader can learn more?



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -571,6 +571,11 @@
     ],
     "sqlState" : "42809"
   },
+  "GENERATED_COLUMN_WITH_DEFAULT_VALUE" : {

Review Comment:
   Nice, thanks for covering the intersection of these two cases!



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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") {

Review Comment:
   you can pattern-match on the error class, e.g.
   
   ```
   case ex: AnalysisException(_, _, _, _, _, "UNRESOLVED_COLUMN.WITH_SUGGESTION", params, _)
   ```
   
   and add the self-reference as a condition, e.g.
   
   ```
   if ex.params.get("objectName").filter(_ == toSQLId(fieldName)).isDefined
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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

Review Comment:
   We ended up banning subquery expressions in column defaults as well. The semantics could be complex for generated columns, you may want to think about whether you want that.



##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -1422,6 +1422,97 @@ 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(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")

Review Comment:
   please check the results of the query here as well?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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(

Review Comment:
   +1



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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))

Review Comment:
   No, it is not.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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 { _ =>

Review Comment:
   is this case-sensitive? Please add a comment, and update the logic if not?



##########
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"

Review Comment:
   for consistency with other metadata keys, we could name this in all capitals with underscores separating words, e.g. "GENERATION_EXPRESSION".



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala:
##########
@@ -2717,4 +2717,21 @@ class DDLParserSuite extends AnalysisTest {
       context = ExpectedContext(
         fragment = "b STRING COMMENT \"abc\" NOT NULL COMMENT \"abc\"", start = 27, stop = 71))
   }
+
+  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,
+        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"),

Review Comment:
   can you add some negative tests where parsing fails as well?



##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -1422,6 +1422,97 @@ 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(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")
+      }
+      // 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 creating generated columns with GENERATED ALWAYS AS expressions"))
+        assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION")
+      }
+    }
+  }
+
+  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") {
+      // 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"))

Review Comment:
   please use `checkError` instead of asserting that the message contains a substring.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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

Review Comment:
   Is this case-sensitive? Please mention in the comment, and update the next line accordingly if not?



##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -1422,6 +1422,97 @@ 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(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")
+      }
+      // 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 creating generated columns with GENERATED ALWAYS AS expressions"))
+        assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION")
+      }
+    }
+  }
+
+  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") {
+      // 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")
+      }
+      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"))
+
+      // Generated column can't reference non-existent column

Review Comment:
   please also add cases where:
   * the expression type is coercible, but not equal to the column type
   * the expression type is not coercible or equal to the column type
   * the expression contains a subquery expression that refers to one of the previous columns
   * the expression contains a subquery expression with two levels of nested subquery expressions
   * the expression contains a subquery expression that scans from another table in the catalog created with a prior CREATE TABLE command
   * the expression refers to one of the columns *after* the `GENERATED ALWAYS AS` column



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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?

Review Comment:
   Yes, we should.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala:
##########
@@ -178,6 +178,15 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val newSchema: StructType =
         ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults(
           schema, tableSpec.provider, "CREATE TABLE", false)
+
+      if (GeneratedColumn.hasGeneratedColumns(newSchema)) {

Review Comment:
   +1, then we can dedup with L214-L220 below



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1442870520

   Looks good for backport.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1442798291

   Are we going to document this new SQL syntax somewhere, @allisonport-db and @cloud-fan ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116513313


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala:
##########
@@ -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(

Review Comment:
   It seems that we missed to define a new error function. I thought we need to define like `generatedColumnsUnsupported` and use it here via `throw QueryCompilationErrors.generatedColumnsUnsupported`. Is there a reason not to define that?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1117384372


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+object GeneratedColumn {
+
+  /**
+   * The metadata key for saving a generation expression in a generated column's metadata. This is

Review Comment:
   for consistency with other metadata keys, we could name this in all capitals with underscores separating words, e.g. "GENERATION_EXPRESSION".



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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 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
+   * - No subquery 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,
+      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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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
+              throw unsupportedExpressionError("generation expression cannot reference itself")
+            }
+            // Check whether the unresolved column is another generated column in the schema
+            schema.filter(isGeneratedColumn).foreach { col =>

Review Comment:
   optional:  you can combine conditions with `exists` like: `if (schema.exists(col => isGeneratedColumn(col) && resolver(...)) { ... }`



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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 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
+   * - No subquery 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,
+      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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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
+              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 (ex.getErrorClass == "UNRESOLVED_ROUTINE") {

Review Comment:
   optional: same here, e.g. `if (ex.errorClass == "UNRESOLVED_ROUTINE" && ex.MessageParameters.get("routineName").isDefined) { ... }`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1066610931


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   I've refactored this PR to instead add `TableCatalogCapability` as suggested. Please let me know what you think.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1049100728


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   Couldn't this be an issue though if we add the syntax to another statement (for example alter table add column) and the catalog/data source (whatever we land on) hasn't updated their implementation yet to accomodate for it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1049102169


##########
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala:
##########
@@ -171,6 +171,12 @@ 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) &&

Review Comment:
   Is there a reason why we match `ResolvedV1Identifier` here instead of `ResolvedV1TableIdentifier`?
   
   I think we would want to add the API to `TableCatalog` and not `CatalogPlugin` right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1336912205

   also cc @huaxingao @sunchao 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116470195


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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)

Review Comment:
   I think it's a bit restrictive to enforce a rule on column order here. We can leave it up to the data source to decide what restrictions to enforce. Valid options include
   - only referencing earlier columns
   - only referencing non-generated columns
   - actually checking for cyclic dependencies
   - etc



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1442786968

   +1 for your backporting decision, @cloud-fan . 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116504025


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java:
##########
@@ -78,6 +80,11 @@ public interface TableCatalog extends CatalogPlugin {
    */
   String OPTION_PREFIX = "option.";
 
+  /**
+   * @return the set of capabilities for this TableCatalog
+   */
+  default Set<TableCatalogCapability> capabilities() {return Collections.emptySet();}

Review Comment:
   Hmm. I thought we have `Java Linter` rule for this. Shall we add more `space` characters?
   ```java
   - {return Collections.emptySet();}
   + { return Collections.emptySet(); }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116517297


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala:
##########
@@ -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(

Review Comment:
   Happy to define a new error function. But I see 
   > 6. Throw the exception with the error class and message parameters. If the same exception is thrown in several places, create an util function in a central place such as QueryCompilationErrors.scala to instantiate the exception
   
   in the error [README](https://github.com/apache/spark/blob/3830285d56d8971c6948763aeb3d99c5bf5eca91/core/src/main/resources/error/README.md) which makes me think it's okay for the one-off usage
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1117619914


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+object GeneratedColumn {
+
+  /**
+   * The metadata key for saving a generation expression in a generated column's metadata. This is

Review Comment:
   Sure works for me



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1039260520


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -1266,6 +1266,11 @@
           "DISTRIBUTE BY clause."
         ]
       },
+      "GENERATED_COLUMN_UNSUPPORTED_FOR_PROVIDER" : {
+        "message" : [
+          "Target data source with table provider: \"<provider>\" does not support creating generated columns with GENERATED ALWAYS AS expressions in <statement>."

Review Comment:
   Can we reuse the `TABLE_OPERATION` error class? The final error message can be `Table xxx does not support generated columns`.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] RussellSpitzer commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1043702758


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   Sounds like it would need to be more like a CatalogCapability? Supports Creating Tables with Generated Columns? 
   
   Although in general i'm not sure how most other engines would be able to support this without supporting Spark Expressions. Do we have a limit on how much of the Spark API is included here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116439411


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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?

Review Comment:
   added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116510001


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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
+   * - 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.
+   */
+  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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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)
+            if (SQLConf.get.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))) {

Review Comment:
   nit. Although this is an exceptional case, shall we define `val resolver` for `SQLConf.get.resolver` to prevent the repetition of `SQLConf.get` invocation? `SQLConf.get` is used at line 130 and 136 here. And, it could be worse in case of **wide** schema with many many columns.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115296490


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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?

Review Comment:
   shall we check if the expression is deterministic? and check if the data type of the expression is compatible with the column type (using `Cast.canUpCast`)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115293788


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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

Review Comment:
   nvm, we already did it by using a special analyzer in default column resolution.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116457847


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -1422,6 +1422,162 @@ 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(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") {
+          sql(s"CREATE TABLE testcat.$tblName(a INT) USING foo")
+        }
+        // Can create table with a generated column
+        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) {
+        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] {

Review Comment:
   why do we comment out this test?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116489013


##########
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.
+ * <p>
+ * A column cannot have both a default value and a generation expression.

Review Comment:
   Just a question. Do we have a proper error message for this case?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116513632


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala:
##########
@@ -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(

Review Comment:
   Is it impossible?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #38823: [SPARK-41290] Support defining generated columns in create table (support GENERATED ALWAYS AS syntax)

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1330964271

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1039265860


##########
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala:
##########
@@ -171,6 +171,12 @@ 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) &&

Review Comment:
   Related to https://github.com/apache/spark/pull/38823/files#r1039264107 . We can easily get the `TableCatalog` here, as `CreateTable.name` is a `ResolvedIdentifier(catalog: TableCatalog, ...)`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1043086633


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   Then I think it's `TableCatalog`'s responsibility to decide if a table supports generated column or not.
   1. For general catalogs like HMS, it may need to resolve the table provider name to `TableProvider`. Then it makes sense to add a new API to `TableProvider`.
   2. For dedicated catalogs like JDBC catalog, it doesn't even have table provider. It depends on the remote SQL database and `TableCatalog` needs a new API to indicate it.
   
   How about we add the new API to both `TableCatalog` and `TableProvider`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1102249114


##########
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

Review Comment:
   Leftover from when I had the metadata key here. Updated



##########
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

Review Comment:
   Leftover from when I had the metadata key here. Removed "values"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116499767


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -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

Review Comment:
   Instead of adding comments, please add explicitly if this is the test assumption.
   ```scala
   withSQLConf(SQLConf.CASE_SENSITIVE.key ->  "false") {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116498689


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -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")

Review Comment:
   nit. `s"` -> `"`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1442786342

   @dongjoon-hyun @huaxingao @sunchao I'd like to include this DS v2 API in 3.4 as this PR has been open for 3 months and this feature is very important to some downstream data sources and users. Are you OK with it? I think this PR is safe to backport as it only adds the API but does not implement it in builtin data sources.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116456581


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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.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.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, 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. 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 */
+  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
+   * - 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.
+   */
+  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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // 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").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 unsupportedExpressionError(s"failed to resolve $fnName to a built-in function")
+          }
+        }
+        throw ex
+    }
+    val analyzed = plan.collectFirst {
+      case Project(Seq(a: Alias), _: LocalRelation) => a.child
+    }.get
+    if (!analyzed.deterministic) {
+      throw unsupportedExpressionError("the expression is not deterministic")
+    }

Review Comment:
   shall we ban subquery expressions?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116483295


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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

Review Comment:
   done, blocked subquery expressions



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116016749


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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

Review Comment:
   Yes. Currently it would just return a generic "function not found" error. Originally I forwarded the original analyzer in there in hopes of returning a more explicit "UDF not supported" error message instead in that case. But we found the code to do that to be very complex, and ended up preferring the separate analyzer instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115287430


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java:
##########
@@ -52,7 +58,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,

Review Comment:
   nit: 4 spaces indentation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116510001


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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
+   * - 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.
+   */
+  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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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)
+            if (SQLConf.get.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))) {

Review Comment:
   nit. Although this is an exceptional case, shall we define `val` for `SQLConf.get.resolver` to prevent the repetition of `SQLConf.get` invocation? `SQLConf.get` is used at line 130 and 136 here. And, it could be worse in case of **wide** schema with many many columns.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
allisonport-db commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1352455011

   > Does the SQL standard say anything about the restrictions of the generate expression? Can we allow `GENERATED AS rand()`? I think at least catalyst should have a rule to check the expression and make sure the data type is the same as column type.
   
   Checking for data type make sense.
   
   Other restrictions we can enforce
   - no UDFs
   - only deterministic functions
   - no subqueries
   - no window functions
   - no aggregate functions
   - no generator functions
   - interdependence with other generated columns


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1098057573


##########
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)
+  }
+}
+
+/**

Review Comment:
   This just uses the `BuiltInFunctionCatalog` defined in `ResolveDefaultColumnsUtil`. We could factor these out into a separate utils to just hold the `BuiltInFunctionCatalog` and a "`BuiltInFuctionAnalyzer`"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1098673813


##########
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

Review Comment:
   what does `and values` mean?



##########
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

Review Comment:
   what does `and values` mean?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements
URL: https://github.com/apache/spark/pull/38823


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115290414


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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

Review Comment:
   cc @dtenedor @gengliangwang shall we also forbid UDF in default value?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115303902


##########
sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala:
##########
@@ -2316,6 +2316,18 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
     }
   }
 
+  test("SPARK-41290: No generated columns with V1") {

Review Comment:
   this is not an insert, shall we move it to `org.apache.spark.sql.execution.command.DDLSuite`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116393855


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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") {

Review Comment:
   Analysis exception isn't a case class.. I can do something like this but I don't think it looks much better
   ```scala
         // Cannot resolve function using built-in catalog
         case ex: AnalysisException if ex.getErrorClass == "UNRESOLVED_ROUTINE" =>
           ex.messageParameters.get("routineName").foreach { fnName =>
             throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function")
           }
           throw ex
         // Generation expression references itself
         case ex: AnalysisException if ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION" &&
             ex.messageParameters.get("objectName")
               .contains(QueryCompilationErrors.toSQLId(fieldName)) =>
           throw unsupportedExpressionError("generation expression cannot reference itself")
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116482692


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala:
##########
@@ -1422,6 +1422,97 @@ 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(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")
+      }
+      // 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 creating generated columns with GENERATED ALWAYS AS expressions"))
+        assert(e.getErrorClass == "UNSUPPORTED_FEATURE.TABLE_OPERATION")
+      }
+    }
+  }
+
+  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") {
+      // 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")
+      }
+      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"))
+
+      // Generated column can't reference non-existent column

Review Comment:
   Added all except the last. See https://github.com/apache/spark/pull/38823#discussion_r1116470195



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116489013


##########
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.
+ * <p>
+ * A column cannot have both a default value and a generation expression.

Review Comment:
   ~Just a question. Do we have a proper error message for this case?~
   Never mind. I found `GENERATED_COLUMN_WITH_DEFAULT_VALUE`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1442793018

   Also cc @viirya , @RussellSpitzer , too.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116549456


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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 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
+   * - No subquery 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,
+      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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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
+              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 (ex.getErrorClass == "UNRESOLVED_ROUTINE") {
+          // Cannot resolve function using built-in catalog
+          ex.messageParameters.get("routineName").foreach { fnName =>
+            throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function")
+          }
+        }
+        throw ex
+    }
+    val analyzed = plan.collectFirst {
+      case Project(Seq(a: Alias), _: LocalRelation) => a.child
+    }.get
+    if (!analyzed.deterministic) {
+      throw unsupportedExpressionError("the expression is not deterministic")

Review Comment:
   ```suggestion
         throw unsupportedExpressionError("generation expression is not deterministic")
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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 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
+   * - No subquery 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,
+      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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result

Review Comment:
   nit:
   ```suggestion
       // Analyze the parsed result
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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 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
+   * - No subquery 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,
+      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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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
+              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 (ex.getErrorClass == "UNRESOLVED_ROUTINE") {
+          // Cannot resolve function using built-in catalog
+          ex.messageParameters.get("routineName").foreach { fnName =>
+            throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function")
+          }
+        }
+        throw ex
+    }
+    val analyzed = plan.collectFirst {
+      case Project(Seq(a: Alias), _: LocalRelation) => a.child
+    }.get
+    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}")

Review Comment:
   ```suggestion
           s"generation expression data type ${analyzed.dataType.simpleString} " +
           s"is incompatible with column data type ${dataType.simpleString}")
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1117631114


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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 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
+   * - No subquery 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,
+      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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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
+              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 (ex.getErrorClass == "UNRESOLVED_ROUTINE") {

Review Comment:
   Didn't do this here since we need to access the value for "routineName"



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.SparkException
+import org.apache.spark.sql.AnalysisException
+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
+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
+ */
+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"
+
+  /** 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 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
+   * - No subquery 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,
+      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)
+    } catch {
+      case ex: ParseException =>
+        // Shouldn't be possible since we check that the expression is a valid catalyst expression
+        // during parsing
+        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}")
+    }
+    // Don't allow subquery expressions
+    if (parsed.containsPattern(PLAN_EXPRESSION)) {
+      throw unsupportedExpressionError("subquery expressions are not allowed for generated columns")
+    }
+    // Analyze the parse result
+    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))
+      analyzer.checkAnalysis(analyzed)
+      analyzed
+    } catch {
+      case ex: AnalysisException =>
+        // 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
+              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 (ex.getErrorClass == "UNRESOLVED_ROUTINE") {

Review Comment:
   Didn't do this here since we need to access the value for "routineName" anyways



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
zsxwing commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1043035473


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   There is no Table object because we haven't created a table before running `CREATE TABLE`. In other words, we only know the table provider name, and we need to know whether this table provider name supports generated columns.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1043667401


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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.
+   *

Review Comment:
   nit: add `<p>` between the paragraphs 



##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java:
##########
@@ -93,4 +93,18 @@ 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() {

Review Comment:
   Yea I'm not sure if the `TableProvider` is a good place for this, since it's only used by data sources who don't have a real catalog. For instance, Iceberg doesn't use it in the general case I think, cc @aokolnychyi @RussellSpitzer 
   
   What about `TableCapability`? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115242394


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java:
##########
@@ -82,6 +98,15 @@ static Column create(
   @Nullable
   ColumnDefaultValue defaultValue();
 
+  /**
+   * Returns the generation expression of this table column. Null means no generation expression.
+   * <p>
+   * 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();

Review Comment:
   I've added it as just a string here as it seems we don't need any additional information



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115294387


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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))

Review Comment:
   is this really useful to expose the original error message to end users?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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))

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1115295605


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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. 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 */
+  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(

Review Comment:
   shall we just rethrow the original exception? We have error context which will point to the exact place in the SQL statement that trigger the error, and we don't need to highlight here that this is a generated expression.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116514685


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala:
##########
@@ -2717,4 +2717,21 @@ class DDLParserSuite extends AnalysisTest {
       context = ExpectedContext(
         fragment = "b STRING COMMENT \"abc\" NOT NULL COMMENT \"abc\"", start = 27, stop = 71))
   }
+
+  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,
+        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"),

Review Comment:
   Added a few



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116515111


##########
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

Review Comment:
   Added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116533658


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala:
##########
@@ -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(

Review Comment:
   Got it. Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonport-db commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "allisonport-db (via GitHub)" <gi...@apache.org>.
allisonport-db commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1117614206


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala:
##########
@@ -136,6 +136,13 @@ case class DataSourceAnalysis(analyzer: Analyzer) extends Rule[LogicalPlan] {
       val newSchema: StructType =
         ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults(
           tableDesc.schema, tableDesc.provider, "CREATE TABLE", false)
+
+      if (GeneratedColumn.hasGeneratedColumns(newSchema)) {
+        throw QueryCompilationErrors.generatedColumnsUnsupported(
+          Seq(tableDesc.identifier.catalog.get, tableDesc.identifier.database.get,
+            tableDesc.identifier.table))
+      }

Review Comment:
   I can add a new error class for it if you think it's necessary?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116504025


##########
sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java:
##########
@@ -78,6 +80,11 @@ public interface TableCatalog extends CatalogPlugin {
    */
   String OPTION_PREFIX = "option.";
 
+  /**
+   * @return the set of capabilities for this TableCatalog
+   */
+  default Set<TableCatalogCapability> capabilities() {return Collections.emptySet();}

Review Comment:
   Hmm. I thought we have `Java Linter` rule for this. Shall we add more `space` characters?
   ```java
   {return Collections.emptySet();}
   { return Collections.emptySet(); }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] chanansh commented on pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "chanansh (via GitHub)" <gi...@apache.org>.
chanansh commented on PR #38823:
URL: https://github.com/apache/spark/pull/38823#issuecomment-1430080148

   when I add data to the table, do I need to explicitly give values to the GENERATED ALWAYS column or is it automatically assigned? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a diff in pull request #38823: [SPARK-41290][SQL] Support GENERATED ALWAYS AS expressions for columns in create/replace table statements

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #38823:
URL: https://github.com/apache/spark/pull/38823#discussion_r1116555041


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala:
##########
@@ -136,6 +136,13 @@ case class DataSourceAnalysis(analyzer: Analyzer) extends Rule[LogicalPlan] {
       val newSchema: StructType =
         ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults(
           tableDesc.schema, tableDesc.provider, "CREATE TABLE", false)
+
+      if (GeneratedColumn.hasGeneratedColumns(newSchema)) {
+        throw QueryCompilationErrors.generatedColumnsUnsupported(
+          Seq(tableDesc.identifier.catalog.get, tableDesc.identifier.database.get,
+            tableDesc.identifier.table))
+      }

Review Comment:
   This can be more specified because it is a V1 table?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org