You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "srielau (via GitHub)" <gi...@apache.org> on 2023/05/01 22:53:21 UTC

[GitHub] [spark] srielau opened a new pull request, #41007: [WIP][SPARK-43205] IDENTIFIER clause

srielau opened a new pull request, #41007:
URL: https://github.com/apache/spark/pull/41007

   <!--
   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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,
+                                          functionArgs: Seq[Expression])
+  extends LeafNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)

Review Comment:
   I only use these for the query ones by rote... I suppose we can change them at will.



-- 
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] gengliangwang commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierClauseUtil.scala:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Resolves the catalog of the name parts for table/view/function/namespace.
+ */
+object IdentifierClauseUtil {
+  private def getNotNullFoldableSring(clauseName: String, expr: Expression): String = {

Review Comment:
   ```suggestion
     private def getNotNullFoldableString(clauseName: String, expr: Expression): String = {
   ```



-- 
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] gengliangwang closed pull request #41007: [SPARK-43205] IDENTIFIER clause

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang closed pull request #41007: [SPARK-43205] IDENTIFIER clause
URL: https://github.com/apache/spark/pull/41007


-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -368,7 +369,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
 
   override def visitDeleteFromTable(
       ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedRelation(ctx.multipartIdentifier())
+    val table = createUnresolvedRelation(ctx.relationReference())

Review Comment:
   This won't work for commands in the legacy style, which does not keep the table/view/namespace as a child logical plan. `CreateTableLikeCommand` is one example, and there can be more in third-party Spark forks.



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -368,7 +369,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
 
   override def visitDeleteFromTable(
       ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedRelation(ctx.multipartIdentifier())
+    val table = createUnresolvedRelation(ctx.relationReference())

Review Comment:
   This is for commands only, and we can do similar things for expressions (column, function, etc.)
   ```
   case class ExprWithUnresolvedIdentifier(identifierExpr: Expression, exprBuilder: Seq[String] => 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] cloud-fan commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2130,8 +2176,11 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     val filter = Option(ctx.where).map(expression(_))
     val ignoreNulls =
       Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false)
-    val function = UnresolvedFunction(
-      getFunctionMultiparts(ctx.functionName), arguments, isDistinct, filter, ignoreNulls)
+    val function = Option(ctx.functionName.expression()).map(p =>
+      UnresolvedFunctionIdentifierClause(expression(p),

Review Comment:
   Shall we add `ExpressionWithUnresolvedIdentifier` to be more general?



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2644,42 +2688,63 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
   }
 
   /**
-   * Create an [[UnresolvedRelation]] from a multi-part identifier context.
+   * Create an [[UnresolvedRelation]] from a multi-part identifier.
    */
   private def createUnresolvedRelation(
-      ctx: MultipartIdentifierContext): UnresolvedRelation = withOrigin(ctx) {
-    UnresolvedRelation(visitMultipartIdentifier(ctx))
+      ctx: ParserRuleContext, ident: Seq[String]): UnresolvedRelation = withOrigin(ctx) {
+    UnresolvedRelation(ident)
   }
 
   /**
-   * Create an [[UnresolvedTable]] from a multi-part identifier context.
+   * Create an [[UnresolvedTable]] from a multi-part identifier.
    */
   private def createUnresolvedTable(
-      ctx: MultipartIdentifierContext,

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] gengliangwang commented on pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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

   @srielau I am rethinking the requirement after reading the related docs (especially [the doc from snowflake](https://docs.snowflake.com/en/sql-reference/identifier-literal))
   So how important it is to support all kinds of expressions(e.g string concats) within the `IDENTIFIER()` clause? It would be much easier if we limit the requirement only accepts the following:
   * quoted identifier
   * session variable
   
   So, instead of 
   ```
   identifierReference
       : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
       | multipartIdentifier
       ;
   ```
   We can make it
   ```
   identifierReference
       : IDENTIFIER_KW LEFT_PAREN '\'' multipartIdentifier  '\'' RIGHT_PAREN
       | IDENTIFIER_KW LEFT_PAREN '"' multipartIdentifier  '"' RIGHT_PAREN
       | multipartIdentifier
       ;
   ```
   And this requires much less changes in the parser and analyzer.
   


-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -402,12 +433,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
 
   override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) {
-    val targetTable = createUnresolvedRelation(ctx.target)
+    val targetTable = withIdentClause(ctx.target,
+      ident => { createUnresolvedRelation(ctx.target, ident) })

Review Comment:
   nit: `val targetTable = withIdentClause(ctx.target, createUnresolvedRelation(ctx.target, _))`



-- 
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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -434,17 +434,31 @@ resource
 dmlStatementNoWith
     : insertInto query                                                             #singleInsertQuery
     | fromClause multiInsertQueryBody+                                             #multiInsertQuery
-    | DELETE FROM multipartIdentifier tableAlias whereClause?                      #deleteFromTable
-    | UPDATE multipartIdentifier tableAlias setClause whereClause?                 #updateTable
-    | MERGE INTO target=multipartIdentifier targetAlias=tableAlias
-        USING (source=multipartIdentifier |
+    | DELETE FROM tableReference tableAlias whereClause?                           #deleteFromTable
+    | UPDATE tableReference tableAlias setClause whereClause?                      #updateTable
+    | MERGE INTO target=tableReference targetAlias=tableAlias
+        USING (source=tableReference |
           LEFT_PAREN sourceQuery=query RIGHT_PAREN) sourceAlias=tableAlias
         ON mergeCondition=booleanExpression
         matchedClause*
         notMatchedClause*
         notMatchedBySourceClause*                                                  #mergeIntoTable
     ;
 
+tableReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+
+identifierReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+identifierReferenceAllowTemp

Review Comment:
   @cloud-fan Can you sketch out what that would look like? What would its visit return a tuple of (Seq[String], 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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -402,101 +424,104 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
 
   override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) {
-    val targetTable = createUnresolvedRelation(ctx.target)
-    val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE")
-    val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable)
-
-    val sourceTableOrQuery = if (ctx.source != null) {
-      createUnresolvedRelation(ctx.source)
-    } else if (ctx.sourceQuery != null) {
-      visitQuery(ctx.sourceQuery)
-    } else {
-      throw QueryParsingErrors.emptySourceForMergeError(ctx)
-    }
-    val sourceTableAlias = getTableAliasWithoutColumnAlias(ctx.sourceAlias, "MERGE")
-    val aliasedSource =
-      sourceTableAlias.map(SubqueryAlias(_, sourceTableOrQuery)).getOrElse(sourceTableOrQuery)
-
-    val mergeCondition = expression(ctx.mergeCondition)
-
-    val matchedActions = ctx.matchedClause().asScala.map {
-      clause => {
-        if (clause.matchedAction().DELETE() != null) {
-          DeleteAction(Option(clause.matchedCond).map(expression))
-        } else if (clause.matchedAction().UPDATE() != null) {
-          val condition = Option(clause.matchedCond).map(expression)
-          if (clause.matchedAction().ASTERISK() != null) {
-            UpdateStarAction(condition)
+    withIdentClause(ctx.target, ident => {

Review Comment:
   MERGE has 2(!) table references. This is the one keep asking about. CREATE TABLE LIKE is another.



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2644,42 +2688,63 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
   }
 
   /**
-   * Create an [[UnresolvedRelation]] from a multi-part identifier context.
+   * Create an [[UnresolvedRelation]] from a multi-part identifier.
    */
   private def createUnresolvedRelation(
-      ctx: MultipartIdentifierContext): UnresolvedRelation = withOrigin(ctx) {
-    UnresolvedRelation(visitMultipartIdentifier(ctx))
+      ctx: ParserRuleContext, ident: Seq[String]): UnresolvedRelation = withOrigin(ctx) {
+    UnresolvedRelation(ident)

Review Comment:
   shall we take `IdentifierReferenceContext` as input and make this function return `LogicalPlan`? Then we can create `PlanWithIdentifier` 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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -1114,6 +1114,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
           case _ => write
         }
 
+      case PlanWithUnresolvedIdentifier(expr, builder) if expr.resolved =>

Review Comment:
   If "we" means @gengliangwang or @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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierClauseUtil.scala:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Resolves the catalog of the name parts for table/view/function/namespace.
+ */
+object IdentifierClauseUtil {
+  private def getNotNullFoldableSring(clauseName: String, expr: Expression): String = {
+    expr match {
+      case e if !e.foldable => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.NOT_CONSTANT",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql))
+      case e if e.dataType != StringType => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.WRONG_TYPE",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql,
+          "dataType" -> e.dataType.catalogString))
+      case e =>
+        e.eval() match {
+          case null => expr.failAnalysis(
+            errorClass = "NOT_A_CONSTANT_STRING.NULL",
+            messageParameters = Map(
+              "name" -> clauseName,
+              "expr" -> expr.sql))
+          case _ => e.eval().toString // OK

Review Comment:
   You mean other.toString, addressed



-- 
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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,

Review Comment:
   Addressed



-- 
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] srielau commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   @dtenedor 
   There are still 30 multipartIdentifier usages that do NOT support IDENTIFIER() notation.
   So we would trade mechanical churn in the grammar for code changes in AstBuilder et al.
   


-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2130,8 +2176,11 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     val filter = Option(ctx.where).map(expression(_))
     val ignoreNulls =
       Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false)
-    val function = UnresolvedFunction(
-      getFunctionMultiparts(ctx.functionName), arguments, isDistinct, filter, ignoreNulls)
+    val function = Option(ctx.functionName.expression()).map(p =>
+      UnresolvedFunctionIdentifierClause(expression(p),

Review Comment:
   And it can be used to replace `UnresolvedAttributeIdentifierClause`



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierClauseUtil.scala:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Resolves the catalog of the name parts for table/view/function/namespace.
+ */
+object IdentifierClauseUtil {
+  private def getNotNullFoldableSring(clauseName: String, expr: Expression): String = {
+    expr match {
+      case e if !e.foldable => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.NOT_CONSTANT",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql))
+      case e if e.dataType != StringType => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.WRONG_TYPE",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql,
+          "dataType" -> e.dataType.catalogString))
+      case e =>
+        e.eval() match {
+          case null => expr.failAnalysis(
+            errorClass = "NOT_A_CONSTANT_STRING.NULL",
+            messageParameters = Map(
+              "name" -> clauseName,
+              "expr" -> expr.sql))
+          case other => other.toString // OK
+        }
+    }
+  }
+
+  def evalIdentifierClause(expr: Expression): Seq[String] = {
+    val str = getNotNullFoldableSring("IDENTIFIER", expr)
+    UnresolvedAttribute.parseAttributeName(str)

Review Comment:
   You mean: UnresolvedAttribute(str).nameParts ?



-- 
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] gengliangwang commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   > @gengliangwang @cloud-fan Can you take a peek with your error-context hat on?
   As you see my adding withOrigin() into the wrapper has focused many of the contexts around the table names.
   Sometimes that is quite right, sometimes that is debatable.
   
   
   So I am seeing code like 
   ```
           withIdentClause(relation, ident => {
             InsertIntoStatement(
               UnresolvedRelation(ident),
               partition,
               cols,
               query,
               overwrite = false,
               ifPartitionNotExists)
           })
   ```
   The context `relation` will be applied on the whole `InsertIntoStatement`. I think what you want is on `UnresolvedRelation(ident)` only. 
   


-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -368,7 +369,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
 
   override def visitDeleteFromTable(
       ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedRelation(ctx.multipartIdentifier())
+    val table = createUnresolvedRelation(ctx.relationReference())

Review Comment:
   The only other idea I can think of is my below alternative to leave the multipartIdentifier references where they are, and just update its definition instead:
   
   ```
   multipartIdentifier
       : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
       | parts+=errorCapturingIdentifier (DOT parts+=errorCapturingIdentifier)*
       ;
   ```
   
   We don't have to do that though to address this particular point, if there is another way.



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -368,7 +369,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
 
   override def visitDeleteFromTable(
       ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedRelation(ctx.multipartIdentifier())
+    val table = createUnresolvedRelation(ctx.relationReference())

Review Comment:
   @cloud-fan Do you want this for commands only, or should I try the same for column, function, table references in queries, INSERT, 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] srielau commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   @gengliangwang @cloud-fan  Can you take a peek with your error-context hat on?
   As you see my adding withOrigin() into the wrapper has focused many of the contexts around the table names.
   Sometimes that is quite right, sometimes that is debatable.
   Did you add all those createUnresolved*() methods for the context? I suppose I can add them back. 


-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3465,10 +3534,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
       properties += PROP_LOCATION -> _
     }
 
-    CreateNamespace(
-      UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
-      ctx.EXISTS != null,
-      properties)
+    withIdentClause(ctx.identifierReference, ident => {

Review Comment:
   Sorry to suggest this at the last minute, but I feel the way we handle MERGE looks pretty neat. How about we change this to
   ```
   CreateNamespace(
     withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)),
     ctx.EXISTS != null,
     properties
   )
   ```
   
   The code diff will be much smaller because it is just a one-line 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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -434,17 +434,31 @@ resource
 dmlStatementNoWith
     : insertInto query                                                             #singleInsertQuery
     | fromClause multiInsertQueryBody+                                             #multiInsertQuery
-    | DELETE FROM multipartIdentifier tableAlias whereClause?                      #deleteFromTable
-    | UPDATE multipartIdentifier tableAlias setClause whereClause?                 #updateTable
-    | MERGE INTO target=multipartIdentifier targetAlias=tableAlias
-        USING (source=multipartIdentifier |
+    | DELETE FROM tableReference tableAlias whereClause?                           #deleteFromTable
+    | UPDATE tableReference tableAlias setClause whereClause?                      #updateTable
+    | MERGE INTO target=tableReference targetAlias=tableAlias
+        USING (source=tableReference |
           LEFT_PAREN sourceQuery=query RIGHT_PAREN) sourceAlias=tableAlias
         ON mergeCondition=booleanExpression
         matchedClause*
         notMatchedClause*
         notMatchedBySourceClause*                                                  #mergeIntoTable
     ;
 
+tableReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+
+identifierReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+identifierReferenceAllowTemp

Review Comment:
   That doesn't work, the context in which they are parsed influences the resolution. E.g. whether they produce a UnresolvedFunctionName, or an UnresolvedRelation, 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] srielau commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   > @srielau I am rethinking the requirement after reading the related docs (especially [the doc from snowflake](https://docs.snowflake.com/en/sql-reference/identifier-literal)) So how important it is to support all kinds of expressions(e.g string concats) within the `IDENTIFIER()` clause? It would be much easier if we limit the requirement only accepts the following:
   > 
   > * quoted identifier
   > * session variable
   > 
   > So, instead of
   > 
   > ```
   > identifierReference
   >     : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
   >     | multipartIdentifier
   >     ;
   > ```
   > 
   > We can make it
   > 
   > ```
   > identifierReference
   >     : IDENTIFIER_KW LEFT_PAREN '\'' multipartIdentifier  '\'' RIGHT_PAREN    #singleQuotedIdentifier
   >     | IDENTIFIER_KW LEFT_PAREN '"' multipartIdentifier  '"' RIGHT_PAREN      #doubleQuotedIdentifier
   >     | IDENTIFIER_KW LEFT_PAREN '$' multipartIdentifier RIGHT_PAREN           #sessionvariableIdentifier
   >     | multipartIdentifier                                                    #simpleMultipartIdentifier
   >     ;
   > ```
   > 
   > And this requires much less changes in the parser and analyzer.
   This is similar to what I tried in my previous attempt: https://github.com/apache/spark/pull/40884
   We need to support parameter markers as well as "proper" (session) variables which would not have a leading '$'.
   They would just be identifiers which need to be resolved.
   Also we need the ability for users to pre-fix and post-fix. For example a schema-name or a field name.
   Without solid support for parameter markers (and variables) we are still open to SQL injection.
   I was told that I cannot call eval from within the parser.
   
     
   


-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala:
##########
@@ -127,23 +127,24 @@ case class UnresolvedFunctionName(
     commandName: String,
     requirePersistent: Boolean,
     funcTypeMismatchHint: Option[String],
-    possibleQualifiedName: Option[Seq[String]] = None) extends LeafNode {

Review Comment:
   Indeed



-- 
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] srielau commented on pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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

   @cloud-fan @gengliangwang @dtenedor Aside from my atrocious Scala skills, the code still needs comments. But It think it's ready for a review.
    


-- 
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] gengliangwang commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -434,17 +434,31 @@ resource
 dmlStatementNoWith
     : insertInto query                                                             #singleInsertQuery
     | fromClause multiInsertQueryBody+                                             #multiInsertQuery
-    | DELETE FROM multipartIdentifier tableAlias whereClause?                      #deleteFromTable
-    | UPDATE multipartIdentifier tableAlias setClause whereClause?                 #updateTable
-    | MERGE INTO target=multipartIdentifier targetAlias=tableAlias
-        USING (source=multipartIdentifier |
+    | DELETE FROM tableReference tableAlias whereClause?                           #deleteFromTable
+    | UPDATE tableReference tableAlias setClause whereClause?                      #updateTable
+    | MERGE INTO target=tableReference targetAlias=tableAlias
+        USING (source=tableReference |
           LEFT_PAREN sourceQuery=query RIGHT_PAREN) sourceAlias=tableAlias
         ON mergeCondition=booleanExpression
         matchedClause*
         notMatchedClause*
         notMatchedBySourceClause*                                                  #mergeIntoTable
     ;
 
+tableReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+
+identifierReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+identifierReferenceAllowTemp

Review Comment:
   +1 with @dtenedor . Shall we have just one identifierReference for all the multipartIdentifier



-- 
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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,
+                                          functionArgs: Seq[Expression])
+  extends LeafNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+}
+
+/**
+* A table-valued function with output column aliases, e.g.
+* {{{
+*   // Assign alias names
+*   select t.a from range(10) t(a);
+* }}}
+*
+* @param identExpr user-specified name of the table-valued function
+* @param child logical plan of the table-valued function
+* @param outputNames alias names of function output columns. The analyzer adds [[Project]]
+*                    to rename the output columns.
+*/
+case class UnresolvedTVFAliasesIdentifierClause(
+                                 identExpr: Expression,

Review Comment:
   addressed



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1492,16 +1520,31 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     } else {
       Seq.empty
     }
-    val name = getFunctionMultiparts(func.functionName)

Review Comment:
   I think we can use `PlanWithIdentifierClause` here
   ```
   withIdentClause(ctx.identifierReference, name => {
     if (name.length > 1) ...
     val tvf = UnresolvedTableValuedFunction(name, func.expression.asScala.map(expression).toSeq)
     val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf
     tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan)
   })
   ```



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala:
##########
@@ -127,23 +127,24 @@ case class UnresolvedFunctionName(
     commandName: String,
     requirePersistent: Boolean,
     funcTypeMismatchHint: Option[String],
-    possibleQualifiedName: Option[Seq[String]] = None) extends LeafNode {

Review Comment:
   changes in this file are unnecessary.



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2644,42 +2688,63 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
   }
 
   /**
-   * Create an [[UnresolvedRelation]] from a multi-part identifier context.
+   * Create an [[UnresolvedRelation]] from a multi-part identifier.
    */
   private def createUnresolvedRelation(
-      ctx: MultipartIdentifierContext): UnresolvedRelation = withOrigin(ctx) {
-    UnresolvedRelation(visitMultipartIdentifier(ctx))
+      ctx: ParserRuleContext, ident: Seq[String]): UnresolvedRelation = withOrigin(ctx) {
+    UnresolvedRelation(ident)

Review Comment:
   Actually MOST of the uses of createRelation() cannot take withIdentClause immediately around UnresolvedRelation.
   See my comments on INSERT INTO



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2644,42 +2688,63 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
   }
 
   /**
-   * Create an [[UnresolvedRelation]] from a multi-part identifier context.
+   * Create an [[UnresolvedRelation]] from a multi-part identifier.
    */
   private def createUnresolvedRelation(
-      ctx: MultipartIdentifierContext): UnresolvedRelation = withOrigin(ctx) {
-    UnresolvedRelation(visitMultipartIdentifier(ctx))
+      ctx: ParserRuleContext, ident: Seq[String]): UnresolvedRelation = withOrigin(ctx) {
+    UnresolvedRelation(ident)

Review Comment:
   Yes, I did wonder about 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] cloud-fan commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -36,6 +36,18 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
 class UnresolvedException(function: String)
   extends AnalysisException(s"Invalid call to $function on unresolved object")
 
+case class PlanWithUnresolvedIdentifier(identifierExpr: Expression,

Review Comment:
   In general, I think it's OK to have multiple ways to hold the unresolved identifier. E.g., `UnresolvedRelationIdentifierClause` is a good solution for clauses that needs `UnresolvedRelation`, which includes FROM, INSERT, DELETE, UPDATE and MERGE. We can use `PlanWithUnresolvedIdentifier` for others so that we don't need to create many `XXXIdentifierClause`.
   
   Shall we use the same idea for expressions(attribute and function)?
   
   



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2130,8 +2176,11 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     val filter = Option(ctx.where).map(expression(_))
     val ignoreNulls =
       Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false)
-    val function = UnresolvedFunction(
-      getFunctionMultiparts(ctx.functionName), arguments, isDistinct, filter, ignoreNulls)
+    val function = Option(ctx.functionName.expression()).map(p =>
+      UnresolvedFunctionIdentifierClause(expression(p),

Review Comment:
   I tried that, but couldn't get it to work. My scala non-skills got the better of me. I will try again.



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2644,42 +2688,63 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
   }
 
   /**
-   * Create an [[UnresolvedRelation]] from a multi-part identifier context.
+   * Create an [[UnresolvedRelation]] from a multi-part identifier.
    */
   private def createUnresolvedRelation(
-      ctx: MultipartIdentifierContext): UnresolvedRelation = withOrigin(ctx) {
-    UnresolvedRelation(visitMultipartIdentifier(ctx))
+      ctx: ParserRuleContext, ident: Seq[String]): UnresolvedRelation = withOrigin(ctx) {
+    UnresolvedRelation(ident)
   }
 
   /**
-   * Create an [[UnresolvedTable]] from a multi-part identifier context.
+   * Create an [[UnresolvedTable]] from a multi-part identifier.
    */
   private def createUnresolvedTable(
-      ctx: MultipartIdentifierContext,

Review Comment:
   Those I could do, also for View and "mostly for TableOrView



-- 
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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,

Review Comment:
   Addressed



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,

Review Comment:
   Addressed



-- 
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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -907,6 +922,7 @@ primaryExpression
     | qualifiedName DOT ASTERISK                                                               #star
     | LEFT_PAREN namedExpression (COMMA namedExpression)+ RIGHT_PAREN                          #rowConstructor
     | LEFT_PAREN query RIGHT_PAREN                                                             #subqueryExpression
+    | IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN                                          #identifierClause

Review Comment:
   What would be the point of 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] cloud-fan commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -402,101 +424,104 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
 
   override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) {
-    val targetTable = createUnresolvedRelation(ctx.target)
-    val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE")
-    val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable)
-
-    val sourceTableOrQuery = if (ctx.source != null) {
-      createUnresolvedRelation(ctx.source)
-    } else if (ctx.sourceQuery != null) {
-      visitQuery(ctx.sourceQuery)
-    } else {
-      throw QueryParsingErrors.emptySourceForMergeError(ctx)
-    }
-    val sourceTableAlias = getTableAliasWithoutColumnAlias(ctx.sourceAlias, "MERGE")
-    val aliasedSource =
-      sourceTableAlias.map(SubqueryAlias(_, sourceTableOrQuery)).getOrElse(sourceTableOrQuery)
-
-    val mergeCondition = expression(ctx.mergeCondition)
-
-    val matchedActions = ctx.matchedClause().asScala.map {
-      clause => {
-        if (clause.matchedAction().DELETE() != null) {
-          DeleteAction(Option(clause.matchedCond).map(expression))
-        } else if (clause.matchedAction().UPDATE() != null) {
-          val condition = Option(clause.matchedCond).map(expression)
-          if (clause.matchedAction().ASTERISK() != null) {
-            UpdateStarAction(condition)
+    withIdentClause(ctx.target, ident => {

Review Comment:
   We are already using `UnresolvedRelationIdentifierClause` for MERGE, there is no need to use `PlanWithIdentifierClause` again.



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1492,16 +1520,31 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     } else {
       Seq.empty
     }
-    val name = getFunctionMultiparts(func.functionName)

Review Comment:
   This may help
   `  def withFuncIdentClause(ctx: FunctionNameContext, builder: Seq[String]
       => LogicalPlan): LogicalPlan = {
       if (ctx.expression != null) {
         PlanWithUnresolvedIdentifier(withOrigin(ctx) { expression(ctx.expression) }, builder)
       } else {
         builder.apply(getFunctionMultiparts(ctx))
       }
     }
   `



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3465,10 +3534,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
       properties += PROP_LOCATION -> _
     }
 
-    CreateNamespace(
-      UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
-      ctx.EXISTS != null,
-      properties)
+    withIdentClause(ctx.identifierReference, ident => {

Review Comment:
   Yeah,  smacked myself. Somehow I thought the withIdentClause must be around the parent node. I'll see where I can do this rework across the board



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierClauseUtil.scala:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Resolves the catalog of the name parts for table/view/function/namespace.
+ */
+object IdentifierClauseUtil {
+  private def getNotNullFoldableSring(clauseName: String, expr: Expression): String = {

Review Comment:
   Yeah, the idea is to make this more broadly available. There are more places where we require a constant string 



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4631,18 +4766,20 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
    * }}}
    */
   override def visitShowColumns(ctx: ShowColumnsContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedTableOrView(ctx.table, "SHOW COLUMNS")
-    val namespace = Option(ctx.ns).map(visitMultipartIdentifier)
-    // Use namespace only if table name doesn't specify it. If namespace is already specified
-    // in the table name, it's checked against the given namespace after table/view is resolved.
-    val tableWithNamespace = if (namespace.isDefined && table.multipartIdentifier.length == 1) {
-      CurrentOrigin.withOrigin(table.origin) {
-        table.copy(multipartIdentifier = namespace.get ++ table.multipartIdentifier)
+    withIdentClause(ctx.table, ident => {
+      val table = createUnresolvedTableOrView(ctx.table, ident, "SHOW COLUMNS")

Review Comment:
   Same here: val tableWithNamespace = if (namespace.isDefined && table.multipartIdentifier.length == 1)
   clobbers that. Only way out would be to push all the logic past analysis?



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -1114,6 +1114,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
           case _ => write
         }
 
+      case PlanWithUnresolvedIdentifier(expr, builder) if expr.resolved =>

Review Comment:
   shall we add a new rule to resolve these identifier related plans/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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
pom.xml:
##########
@@ -170,12 +170,8 @@
     <scala.version>2.12.17</scala.version>
     <scala.binary.version>2.12</scala.binary.version>
     <scalatest-maven-plugin.version>2.2.0</scalatest-maven-plugin.version>
-    <!--
-      This needs to be managed in different profiles to avoid
-      errors building different Hadoop versions.
-      See: SPARK-36547, SPARK-38394.
-       -->
-    <scala-maven-plugin.version>4.8.1</scala-maven-plugin.version>
+    <!-- dont update scala-maven-plugin to version 4.8.1 SPARK-42809 and SPARK-43595 -->   
+    <scala-maven-plugin.version>4.8.0</scala-maven-plugin.version>

Review Comment:
   ```suggestion
       <!--
         This needs to be managed in different profiles to avoid
         errors building different Hadoop versions.
         See: SPARK-36547, SPARK-38394.
          -->
       <scala-maven-plugin.version>4.8.1</scala-maven-plugin.version>
   ```



-- 
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] gengliangwang commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierClauseUtil.scala:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Resolves the catalog of the name parts for table/view/function/namespace.
+ */
+object IdentifierClauseUtil {
+  private def getNotNullFoldableSring(clauseName: String, expr: Expression): String = {

Review Comment:
   It seems that the first parameter is always `IDENTIFIER`?



-- 
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 #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -434,17 +434,31 @@ resource
 dmlStatementNoWith
     : insertInto query                                                             #singleInsertQuery
     | fromClause multiInsertQueryBody+                                             #multiInsertQuery
-    | DELETE FROM multipartIdentifier tableAlias whereClause?                      #deleteFromTable
-    | UPDATE multipartIdentifier tableAlias setClause whereClause?                 #updateTable
-    | MERGE INTO target=multipartIdentifier targetAlias=tableAlias
-        USING (source=multipartIdentifier |
+    | DELETE FROM tableReference tableAlias whereClause?                           #deleteFromTable
+    | UPDATE tableReference tableAlias setClause whereClause?                      #updateTable
+    | MERGE INTO target=tableReference targetAlias=tableAlias
+        USING (source=tableReference |
           LEFT_PAREN sourceQuery=query RIGHT_PAREN) sourceAlias=tableAlias
         ON mergeCondition=booleanExpression
         matchedClause*
         notMatchedClause*
         notMatchedBySourceClause*                                                  #mergeIntoTable
     ;
 
+tableReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+
+identifierReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+identifierReferenceAllowTemp

Review Comment:
   This does not seem any different than `identifierReference` above, or `tableReference`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierClauseUtil.scala:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Resolves the catalog of the name parts for table/view/function/namespace.
+ */
+object IdentifierClauseUtil {
+  private def getNotNullFoldableSring(clauseName: String, expr: Expression): String = {
+    expr match {
+      case e if !e.foldable => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.NOT_CONSTANT",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql))
+      case e if e.dataType != StringType => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.WRONG_TYPE",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql,
+          "dataType" -> e.dataType.catalogString))
+      case e =>
+        e.eval() match {
+          case null => expr.failAnalysis(
+            errorClass = "NOT_A_CONSTANT_STRING.NULL",
+            messageParameters = Map(
+              "name" -> clauseName,
+              "expr" -> expr.sql))
+          case _ => e.eval().toString // OK

Review Comment:
   this makes us perform the `eval` again. I think we can just write `case other => other // OK` and it should work to reuse it instead.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {

Review Comment:
   optional: it might be simpler to just add this as another constructor inside the above case class instead of creating a separate helper object for this purpose?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,
+                                          functionArgs: Seq[Expression])
+  extends LeafNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+}
+
+/**
+* A table-valued function with output column aliases, e.g.
+* {{{
+*   // Assign alias names
+*   select t.a from range(10) t(a);
+* }}}
+*
+* @param identExpr user-specified name of the table-valued function
+* @param child logical plan of the table-valued function
+* @param outputNames alias names of function output columns. The analyzer adds [[Project]]
+*                    to rename the output columns.
+*/
+case class UnresolvedTVFAliasesIdentifierClause(
+                                 identExpr: Expression,
+                                 child: LogicalPlan,
+                                 outputNames: Seq[String]) extends UnaryNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan =
+    copy(child = newChild)
+}
+
+/**
+ * Holds the name of a relation that has yet to be looked up in a catalog.
+ *
+ * @param expr multipartIdentifier table name
+ */
+case class UnresolvedRelationIdentifierClause(expr: Expression)
+  extends UnresolvedRelationOrIdentifierClause {
+
+  /** Returns a `.` separated name for this relation. */
+  def tableName: String = throw new UnresolvedException("tableName")

Review Comment:
   QQ: why do we need this method if it just throws an exception and does not override anything?



##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -907,6 +922,7 @@ primaryExpression
     | qualifiedName DOT ASTERISK                                                               #star
     | LEFT_PAREN namedExpression (COMMA namedExpression)+ RIGHT_PAREN                          #rowConstructor
     | LEFT_PAREN query RIGHT_PAREN                                                             #subqueryExpression
+    | IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN                                          #identifierClause

Review Comment:
   we might be OK with just this change in the grammar file to introduce the new IDENTIFIER clause feature, and not any of the other changes? We could always still do the other changes as well, but it could behoove us to split that into a separate PR?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,

Review Comment:
   please fix indentation?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,
+                                          functionArgs: Seq[Expression])
+  extends LeafNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+}
+
+/**
+* A table-valued function with output column aliases, e.g.
+* {{{
+*   // Assign alias names
+*   select t.a from range(10) t(a);
+* }}}
+*
+* @param identExpr user-specified name of the table-valued function
+* @param child logical plan of the table-valued function
+* @param outputNames alias names of function output columns. The analyzer adds [[Project]]
+*                    to rename the output columns.
+*/
+case class UnresolvedTVFAliasesIdentifierClause(
+                                 identExpr: Expression,

Review Comment:
   please fix indentation?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala:
##########
@@ -40,18 +40,34 @@ case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNod
   override def output: Seq[Attribute] = Nil
 }
 
+abstract class UnresolvedTableOrIdentifierClause extends LeafNode
+
 /**
  * Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to
  * [[ResolvedTable]] during analysis.
  */
 case class UnresolvedTable(
     multipartIdentifier: Seq[String],
     commandName: String,
-    relationTypeMismatchHint: Option[String]) extends LeafNode {
+    relationTypeMismatchHint: Option[String]) extends UnresolvedTableOrIdentifierClause {
   override lazy val resolved: Boolean = false
 
   override def output: Seq[Attribute] = Nil
 }
+/**
+ * Holds the identifier of a tabl ein String form that has yet to be looked up in a catalog.
+ * It will be resolved to [[UnResolvedTable]] during analysis.
+ */
+case class UnresolvedTableIdentifierClause(

Review Comment:
   should this just be in `unresolved.scala` next to the other new unresolved identifier clause nodes? Same for the others below.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -2108,6 +2123,18 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
           }
         }
 
+      // Resolve IDENTIFIER clause.
+      case u@UnresolvedRelationIdentifierClause(expr) if expr.resolved =>
+        UnresolvedRelation(IdentifierClauseUtil.evalIdentifierClause(expr))
+
+      // Resolve IDENTIFIER clause and reduce to a regular unresolved table function
+      case u@UnresolvedTableValuedFunctionIdentifierClause(expr, args) if expr.resolved =>
+        UnresolvedTableValuedFunction(IdentifierClauseUtil.evalIdentifierClause(expr), args)
+
+      // Resolve IDENTIFIER clause and reduce to a regular unresolved TVF aliases

Review Comment:
   ```suggestion
         // Resolve IDENTIFIER clause and reduce to a regular unresolved TVF alias
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,
+                                          functionArgs: Seq[Expression])
+  extends LeafNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)

Review Comment:
   optional: do we want to create separate patterns for each of these types of unresolved identifier clause? Or just combine them all into one? Having separate ones could grant more flexibility later if needed?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,

Review Comment:
   please fix indentation?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,

Review Comment:
   please fix indentation?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala:
##########
@@ -40,18 +40,34 @@ case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNod
   override def output: Seq[Attribute] = Nil
 }
 
+abstract class UnresolvedTableOrIdentifierClause extends LeafNode
+
 /**
  * Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to
  * [[ResolvedTable]] during analysis.
  */
 case class UnresolvedTable(
     multipartIdentifier: Seq[String],
     commandName: String,
-    relationTypeMismatchHint: Option[String]) extends LeafNode {
+    relationTypeMismatchHint: Option[String]) extends UnresolvedTableOrIdentifierClause {
   override lazy val resolved: Boolean = false
 
   override def output: Seq[Attribute] = Nil
 }
+/**
+ * Holds the identifier of a tabl ein String form that has yet to be looked up in a catalog.

Review Comment:
   ```suggestion
    * Holds the identifier of a table in string form that has yet to be looked up in a catalog.
   ```



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala:
##########
@@ -40,18 +40,34 @@ case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNod
   override def output: Seq[Attribute] = Nil
 }
 
+abstract class UnresolvedTableOrIdentifierClause extends LeafNode
+
 /**
  * Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to
  * [[ResolvedTable]] during analysis.
  */
 case class UnresolvedTable(
     multipartIdentifier: Seq[String],
     commandName: String,
-    relationTypeMismatchHint: Option[String]) extends LeafNode {
+    relationTypeMismatchHint: Option[String]) extends UnresolvedTableOrIdentifierClause {
   override lazy val resolved: Boolean = false
 
   override def output: Seq[Attribute] = Nil
 }
+/**
+ * Holds the identifier of a tabl ein String form that has yet to be looked up in a catalog.
+ * It will be resolved to [[UnResolvedTable]] during analysis.
+ */
+case class UnresolvedTableIdentifierClause(

Review Comment:
   I wondered why the related nodes (such as UnresolvedView) are in this file. If we move the new objects to unresolved.scala, we should move the existing ones as well. @cloud-fan @gengliangwang ?



-- 
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 pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   @srielau @gengliangwang @cloud-fan 
   
   The general structure of the PR looks OK.
   
   This PR proposes to add new unresolved nodes for several different locations in the parser. This could be useful in the future if we want to add more custom analysis support for these different areas.
   
   Alternatively, we could leave the multipartIdentifier references where they are, and just update its definition instead:
   
   ```
   multipartIdentifier
       : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
       | parts+=errorCapturingIdentifier (DOT parts+=errorCapturingIdentifier)*
       ;
   ```
   
   This would reduce the number of changes to the .g4 file, since the proposed namespaceReference and functionNameReference and relationReference are all the same syntax. Then the PR would be easier to merge into different Spark forks out there. But this approach grants us future flexibility, if we ever anticipate the syntax to diverge for these different cases. I am OK with the proposed approach 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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -368,7 +369,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
 
   override def visitDeleteFromTable(
       ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedRelation(ctx.multipartIdentifier())
+    val table = createUnresolvedRelation(ctx.relationReference())

Review Comment:
   Right now visitMultipartIdentifer returns a Seq[String]. This change would force a change. I fear that is the change Wenchen is worried about (?).
   Also, note that there are still 30 multipartIdentifier usages remaining which do NOT support IDENTIFIER() notation.
   So I think we would trade mechanical churn in the grammar for code changes in AstBuilder et al. 
   



-- 
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] srielau commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   @gengliangwang @dtenedor @gengliangwang 
   It's camera ready now. Please have at 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] cloud-fan commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4545,28 +4672,32 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
     val query = Option(ctx.query).map(plan)
-    val relation = createUnresolvedRelation(ctx.multipartIdentifier)

Review Comment:
   shall we only wrap the relation with `withIdentClause`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4631,18 +4766,20 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
    * }}}
    */
   override def visitShowColumns(ctx: ShowColumnsContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedTableOrView(ctx.table, "SHOW COLUMNS")
-    val namespace = Option(ctx.ns).map(visitMultipartIdentifier)
-    // Use namespace only if table name doesn't specify it. If namespace is already specified
-    // in the table name, it's checked against the given namespace after table/view is resolved.
-    val tableWithNamespace = if (namespace.isDefined && table.multipartIdentifier.length == 1) {
-      CurrentOrigin.withOrigin(table.origin) {
-        table.copy(multipartIdentifier = namespace.get ++ table.multipartIdentifier)
+    withIdentClause(ctx.table, ident => {
+      val table = createUnresolvedTableOrView(ctx.table, ident, "SHOW COLUMNS")

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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2111,6 +2143,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Create a IDENTIFIER() clause.
+   */
+  override def visitIdentifierClause(ctx: IdentifierClauseContext): Expression = withOrigin(ctx) {
+    // Create the function call.
+    val expr : Expression = expression(ctx.expression())

Review Comment:
   ```suggestion
       val expr: Expression = expression(ctx.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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -402,12 +433,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
 
   override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) {
-    val targetTable = createUnresolvedRelation(ctx.target)
+    val targetTable = withIdentClause(ctx.target,
+      ident => { createUnresolvedRelation(ctx.target, ident) })

Review Comment:
   Done, also for source



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala:
##########
@@ -36,6 +36,8 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
         val CatalogAndIdentifier(catalog, identifier) = nameParts
         ResolvedIdentifier(catalog, identifier)
       }
+    case UnresolvedIdentifierClause(expr, allowTemp) if (expr.resolved) =>
+      UnresolvedIdentifier(IdentifierClauseUtil.evalIdentifierClause(expr), allowTemp)

Review Comment:
   This appears to be dead code. Removed
   



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3465,10 +3534,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
       properties += PROP_LOCATION -> _
     }
 
-    CreateNamespace(
-      UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
-      ctx.EXISTS != null,
-      properties)
+    withIdentClause(ctx.identifierReference, ident => {

Review Comment:
   Done, I'm now wondering whether I can bury it in the createUnresolved* functions, make it largely invisible,
   



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {
+  def apply(
+             identExpr: Expression,
+             arguments: Seq[Expression],
+             isDistinct: Boolean): UnresolvedFunction = {
+    UnresolvedFunctionIdentifierClause(identExpr, arguments, isDistinct)
+  }
+}
+
+/**
+ * A table-valued function, e.g.
+ * {{{
+ *   select id from range(10);
+ * }}}
+ *
+ * @param identExpr user-specified expression with teh name of this table-value function
+ * @param functionArgs list of function arguments
+ */
+case class UnresolvedTableValuedFunctionIdentifierClause(
+                                          identExpr: Expression,
+                                          functionArgs: Seq[Expression])
+  extends LeafNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+}
+
+/**
+* A table-valued function with output column aliases, e.g.
+* {{{
+*   // Assign alias names
+*   select t.a from range(10) t(a);
+* }}}
+*
+* @param identExpr user-specified name of the table-valued function
+* @param child logical plan of the table-valued function
+* @param outputNames alias names of function output columns. The analyzer adds [[Project]]
+*                    to rename the output columns.
+*/
+case class UnresolvedTVFAliasesIdentifierClause(
+                                 identExpr: Expression,
+                                 child: LogicalPlan,
+                                 outputNames: Seq[String]) extends UnaryNode {
+
+  override def output: Seq[Attribute] = Nil
+
+  override lazy val resolved = false
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan =
+    copy(child = newChild)
+}
+
+/**
+ * Holds the name of a relation that has yet to be looked up in a catalog.
+ *
+ * @param expr multipartIdentifier table name
+ */
+case class UnresolvedRelationIdentifierClause(expr: Expression)
+  extends UnresolvedRelationOrIdentifierClause {
+
+  /** Returns a `.` separated name for this relation. */
+  def tableName: String = throw new UnresolvedException("tableName")

Review Comment:
   Yeah this was monkey see, monkey do poorly. Removed



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +279,138 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = Seq(expr)
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = "IDENTIFIER"
+  override def toString: String = {
+    s"'(${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+  UnresolvedAttributeIdentifierClause = {
+      copy(expr = newChildren.head)
+  }
+}
+
+case class UnresolvedFunctionIdentifierClause(
+                               identExpr: Expression,
+                               arguments: Seq[Expression],
+                               isDistinct: Boolean,
+                               filter: Option[Expression] = None,
+                               ignoreNulls: Boolean = false)
+  extends Expression with Unevaluable {
+
+  override def children: Seq[Expression] = arguments ++ filter.toSeq
+
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  override lazy val resolved = false
+  final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_CLAUSE)
+
+  override def prettyName: String = identExpr.toString
+  override def toString: String = {
+    val distinct = if (isDistinct) "distinct " else ""
+    s"'${identExpr.toString}($distinct${children.mkString(", ")})"
+  }
+
+  override protected def withNewChildrenInternal
+  (newChildren: IndexedSeq[Expression]): UnresolvedFunctionIdentifierClause = {
+    if (filter.isDefined) {
+      copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last))
+    } else {
+      copy(arguments = newChildren)
+    }
+  }
+}
+
+object UnresolvedFunctionIdentifierClause {

Review Comment:
   I think this i unused actually - removed



-- 
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 #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -434,17 +434,31 @@ resource
 dmlStatementNoWith
     : insertInto query                                                             #singleInsertQuery
     | fromClause multiInsertQueryBody+                                             #multiInsertQuery
-    | DELETE FROM multipartIdentifier tableAlias whereClause?                      #deleteFromTable
-    | UPDATE multipartIdentifier tableAlias setClause whereClause?                 #updateTable
-    | MERGE INTO target=multipartIdentifier targetAlias=tableAlias
-        USING (source=multipartIdentifier |
+    | DELETE FROM tableReference tableAlias whereClause?                           #deleteFromTable
+    | UPDATE tableReference tableAlias setClause whereClause?                      #updateTable
+    | MERGE INTO target=tableReference targetAlias=tableAlias
+        USING (source=tableReference |
           LEFT_PAREN sourceQuery=query RIGHT_PAREN) sourceAlias=tableAlias
         ON mergeCondition=booleanExpression
         matchedClause*
         notMatchedClause*
         notMatchedBySourceClause*                                                  #mergeIntoTable
     ;
 
+tableReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+
+identifierReference
+    : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
+    | multipartIdentifier
+    ;
+identifierReferenceAllowTemp

Review Comment:
   Or we can ask the caller side (e.g. the parser of CREATE TABLE command) to create the proper identifier logical plan, but I think using antlr rules to track the context is simpler.
   
   BTW I think we can still have a `multipartIdentifierOrClause` to avoid duplicated code.



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierClauseUtil.scala:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Resolves the catalog of the name parts for table/view/function/namespace.
+ */
+object IdentifierClauseUtil {
+  private def getNotNullFoldableSring(clauseName: String, expr: Expression): String = {
+    expr match {
+      case e if !e.foldable => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.NOT_CONSTANT",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql))
+      case e if e.dataType != StringType => expr.failAnalysis(
+        errorClass = "NOT_A_CONSTANT_STRING.WRONG_TYPE",
+        messageParameters = Map(
+          "name" -> clauseName,
+          "expr" -> expr.sql,
+          "dataType" -> e.dataType.catalogString))
+      case e =>
+        e.eval() match {
+          case null => expr.failAnalysis(
+            errorClass = "NOT_A_CONSTANT_STRING.NULL",
+            messageParameters = Map(
+              "name" -> clauseName,
+              "expr" -> expr.sql))
+          case other => other.toString // OK
+        }
+    }
+  }
+
+  def evalIdentifierClause(expr: Expression): Seq[String] = {
+    val str = getNotNullFoldableSring("IDENTIFIER", expr)
+    UnresolvedAttribute.parseAttributeName(str)

Review Comment:
   ```suggestion
       UnresolvedAttribute(str)
   ```
   The `apply` method invokes the sql parser to parse the name, which is more standard. `parseAttributeName` uses a hand-written small parser.



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4687,9 +4777,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
     AddPartitions(
       createUnresolvedTable(
-        ctx.multipartIdentifier,
-        "ALTER TABLE ... ADD PARTITION ...",
-        alterTableTypeMismatchHint),
+          ctx.identifierReference,
+          "ALTER TABLE ... ADD PARTITION ...",
+          alterTableTypeMismatchHint),

Review Comment:
   ```suggestion
           ctx.identifierReference,
           "ALTER TABLE ... ADD PARTITION ...",
           alterTableTypeMismatchHint),
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4705,10 +4795,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
   override def visitRenameTablePartition(
       ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) {
     RenamePartitions(
-      createUnresolvedTable(
-        ctx.multipartIdentifier,
-        "ALTER TABLE ... RENAME TO PARTITION",
-        alterTableTypeMismatchHint),
+       createUnresolvedTable(
+          ctx.identifierReference,
+          "ALTER TABLE ... RENAME TO PARTITION",
+          alterTableTypeMismatchHint),

Review Comment:
   ```suggestion
         createUnresolvedTable(
           ctx.identifierReference,
           "ALTER TABLE ... RENAME TO PARTITION",
           alterTableTypeMismatchHint),
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4848,40 +4951,45 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
         case Some(x) => throw QueryParsingErrors.showFunctionsUnsupportedError(x, ctx.identifier())
     }
 
-    val ns = Option(ctx.ns).map(visitMultipartIdentifier)
     val legacy = Option(ctx.legacy).map(visitMultipartIdentifier)
-    val nsPlan = if (ns.isDefined) {
+    val pattern = Option(ctx.pattern).map(x => string(visitStringLit(x))).orElse(legacy.map(_.last))
+
+    if (ctx.ns != null) {
       if (legacy.isDefined) {
         throw QueryParsingErrors.showFunctionsInvalidPatternError(ctx.legacy.getText, ctx.legacy)
       }
-      UnresolvedNamespace(ns.get)
+      ShowFunctions(
+        withIdentClause(ctx.ns, UnresolvedNamespace(_)),
+        userScope, systemScope, pattern)
     } else if (legacy.isDefined) {
-      UnresolvedNamespace(legacy.get.dropRight(1))
+        ShowFunctions(UnresolvedNamespace(legacy.get.dropRight(1)), userScope, systemScope, pattern)
     } else {
-      UnresolvedNamespace(Nil)
+        ShowFunctions(UnresolvedNamespace(Nil), userScope, systemScope, pattern)
     }
-    val pattern = Option(ctx.pattern).map(x => string(visitStringLit(x))).orElse(legacy.map(_.last))
-    ShowFunctions(nsPlan, userScope, systemScope, pattern)
   }
 
   override def visitRefreshFunction(ctx: RefreshFunctionContext): LogicalPlan = withOrigin(ctx) {
-    val functionIdentifier = visitMultipartIdentifier(ctx.multipartIdentifier)
-    RefreshFunction(UnresolvedFunctionName(
-      functionIdentifier,
-      "REFRESH FUNCTION",
-      requirePersistent = true,
-      funcTypeMismatchHint = None))
+    RefreshFunction(
+      withIdentClause(
+        ctx.identifierReference,
+        createUnresolvedFunctionName(
+          ctx.identifierReference,
+          _,
+          "REFRESH FUNCTION",
+          requirePersistent = true,
+          funcTypeMismatchHint = None)))
   }
 
   override def visitCommentNamespace(ctx: CommentNamespaceContext): LogicalPlan = withOrigin(ctx) {
     val comment = visitComment(ctx.comment)
-    val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier)
-    CommentOnNamespace(UnresolvedNamespace(nameParts), comment)
+    CommentOnNamespace(withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), comment)
   }
 
   override def visitCommentTable(ctx: CommentTableContext): LogicalPlan = withOrigin(ctx) {
     val comment = visitComment(ctx.comment)
-    CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
+      CommentOnTable(
+        createUnresolvedTable(ctx.identifierReference, "COMMENT ON TABLE"),
+        comment)

Review Comment:
   ```suggestion
       CommentOnTable(createUnresolvedTable(ctx.identifierReference, "COMMENT ON TABLE"), comment)
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4848,40 +4951,45 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
         case Some(x) => throw QueryParsingErrors.showFunctionsUnsupportedError(x, ctx.identifier())
     }
 
-    val ns = Option(ctx.ns).map(visitMultipartIdentifier)
     val legacy = Option(ctx.legacy).map(visitMultipartIdentifier)
-    val nsPlan = if (ns.isDefined) {
+    val pattern = Option(ctx.pattern).map(x => string(visitStringLit(x))).orElse(legacy.map(_.last))
+
+    if (ctx.ns != null) {
       if (legacy.isDefined) {
         throw QueryParsingErrors.showFunctionsInvalidPatternError(ctx.legacy.getText, ctx.legacy)
       }
-      UnresolvedNamespace(ns.get)
+      ShowFunctions(
+        withIdentClause(ctx.ns, UnresolvedNamespace(_)),
+        userScope, systemScope, pattern)
     } else if (legacy.isDefined) {
-      UnresolvedNamespace(legacy.get.dropRight(1))
+        ShowFunctions(UnresolvedNamespace(legacy.get.dropRight(1)), userScope, systemScope, pattern)

Review Comment:
   ```suggestion
         ShowFunctions(UnresolvedNamespace(legacy.get.dropRight(1)), userScope, systemScope, pattern)
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4848,40 +4951,45 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
         case Some(x) => throw QueryParsingErrors.showFunctionsUnsupportedError(x, ctx.identifier())
     }
 
-    val ns = Option(ctx.ns).map(visitMultipartIdentifier)
     val legacy = Option(ctx.legacy).map(visitMultipartIdentifier)
-    val nsPlan = if (ns.isDefined) {
+    val pattern = Option(ctx.pattern).map(x => string(visitStringLit(x))).orElse(legacy.map(_.last))
+
+    if (ctx.ns != null) {
       if (legacy.isDefined) {
         throw QueryParsingErrors.showFunctionsInvalidPatternError(ctx.legacy.getText, ctx.legacy)
       }
-      UnresolvedNamespace(ns.get)
+      ShowFunctions(
+        withIdentClause(ctx.ns, UnresolvedNamespace(_)),
+        userScope, systemScope, pattern)
     } else if (legacy.isDefined) {
-      UnresolvedNamespace(legacy.get.dropRight(1))
+        ShowFunctions(UnresolvedNamespace(legacy.get.dropRight(1)), userScope, systemScope, pattern)
     } else {
-      UnresolvedNamespace(Nil)
+        ShowFunctions(UnresolvedNamespace(Nil), userScope, systemScope, pattern)

Review Comment:
   ```suggestion
         ShowFunctions(UnresolvedNamespace(Nil), userScope, systemScope, pattern)
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4657,9 +4747,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
       ctx: RecoverPartitionsContext): LogicalPlan = withOrigin(ctx) {
     RecoverPartitions(
       createUnresolvedTable(
-        ctx.multipartIdentifier,
-        "ALTER TABLE ... RECOVER PARTITIONS",
-        alterTableTypeMismatchHint))
+          ctx.identifierReference,
+          "ALTER TABLE ... RECOVER PARTITIONS",
+          alterTableTypeMismatchHint))

Review Comment:
   ```suggestion
           ctx.identifierReference,
           "ALTER TABLE ... RECOVER PARTITIONS",
           alterTableTypeMismatchHint))
   ```



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +315,99 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)

Review Comment:
   It's used in visitIdentifierClause. What I always need clarification on is whether visit* routines are invoked automatically or not... identifierClause is used in primaryExpression (basically a column reference in an 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] cloud-fan commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala:
##########
@@ -36,6 +36,8 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
         val CatalogAndIdentifier(catalog, identifier) = nameParts
         ResolvedIdentifier(catalog, identifier)
       }
+    case UnresolvedIdentifierClause(expr, allowTemp) if (expr.resolved) =>
+      UnresolvedIdentifier(IdentifierClauseUtil.evalIdentifierClause(expr), allowTemp)

Review Comment:
   `UnresolvedIdentifier` is a logical plan, can we use `PlanWithUnresolvedIdentifier` 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] cloud-fan commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -36,6 +36,18 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
 class UnresolvedException(function: String)
   extends AnalysisException(s"Invalid call to $function on unresolved object")
 
+case class PlanWithUnresolvedIdentifier(identifierExpr: Expression,

Review Comment:
   In general, I think it's OK to have multiple ways to hold the unresolved identifier. E.g., `UnresolvedRelationIdentifierClause` is a good solution for clauses that needs `UnresolvedRelation`, which includes FROM, INSERT, UPDATE and MERGE. We can use `PlanWithUnresolvedIdentifier` for others so that we don't need to create many `XXXIdentifierClause`.
   
   Shall we use the same idea for expressions(attribute and function)?
   
   



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -36,6 +36,18 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
 class UnresolvedException(function: String)
   extends AnalysisException(s"Invalid call to $function on unresolved object")
 
+case class PlanWithUnresolvedIdentifier(identifierExpr: Expression,

Review Comment:
   In general, I think it's OK to have multiple ways to hold the unresolved identifier. E.g., `UnresolvedRelationIdentifierClause` is a good solution for plans that needs `UnresolvedRelation`, which includes FROM, INSERT, UPDATE and MERGE. We can use `PlanWithUnresolvedIdentifier` for others so that we don't need to create many `XXXIdentifierClause`.
   
   Shall we use the same idea for expressions(attribute and function)?
   
   



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -272,27 +282,33 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     ctx match {
       case table: InsertIntoTableContext =>
         val (relation, cols, partition, ifPartitionNotExists) = visitInsertIntoTable(table)
-        InsertIntoStatement(
-          relation,
-          partition,
-          cols,
-          query,
-          overwrite = false,
-          ifPartitionNotExists)
+        withIdentClause(relation, ident => {
+          InsertIntoStatement(

Review Comment:
   since we already have `UnresolvedRelationIdentifierClause`, we can use it for INSERT.



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -368,7 +369,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
 
   override def visitDeleteFromTable(
       ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedRelation(ctx.multipartIdentifier())
+    val table = createUnresolvedRelation(ctx.relationReference())

Review Comment:
   We can add a new parser rule as this PR already did
   ```
   identifierReference:
       : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN
       | multipartIdentifier
   ```
   
   then we update the parser rules for commands one by one, for each command, we update the corresponding scala part. Taking DROP TABLE as an example, the current scala part is
   ```
     override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
       // DROP TABLE works with either a table or a temporary view.
       DropTable(
         UnresolvedIdentifier(visitMultipartIdentifier(ctx.multipartIdentifier()), allowTemp = true),
         ctx.EXISTS != null,
         ctx.PURGE != null)
     }
   ```
   we can change it to
   ```
   case class PlanWithUnresolvedIdentifier(identifierExpr: Expression, planBuilder: Seq[String] => LogicalPlan)
   
   def withIdentClause(ctx: IdentifierReferenceContext, builder: Seq[String] => LogicalPlan): LogicalPlan = {
     if (ctx.expression != null) {
       PlanWithUnresolvedIdentifier(expr(ctx.expression), builder)
     } else {
       builder.apply(visitMultipartIdentifier(ctx.multipartIdentifier()))
     }
   }
   
   override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
     withIdentClause(ctx.identifierReference, ident => {
           DropTable(
             UnresolvedIdentifier(ident, allowTemp = true),
             ctx.EXISTS != null,
             ctx.PURGE != null)
     }) 
   }
   ```



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -368,7 +369,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
 
   override def visitDeleteFromTable(
       ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
-    val table = createUnresolvedRelation(ctx.multipartIdentifier())
+    val table = createUnresolvedRelation(ctx.relationReference())

Review Comment:
   OK, alternatives?
   



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -1261,6 +1268,29 @@
     ],
     "sqlState" : "42000"
   },
+  "NOT_A_CONSTANT_STRING" : {
+    "message" : [
+      "The expression <expr> used for the routine or clause <name> must be a constant STRING which is NOT NULL."
+    ],
+    "subClass" : {
+      "NOT_CONSTANT" : {
+        "message" : [
+          "To be considered constant the expression must not depend on any columns, contain a subquery, or invoke a non deterministic function such as rand()."
+        ]
+      },
+      "NULL" : {
+        "message" : [
+          "The expression evaluate to NULL."

Review Comment:
   ```suggestion
             "The expression evaluates to NULL."
   ```



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -1156,6 +1156,13 @@
       "The url is invalid: <url>. If necessary set <ansiConfig> to \"false\" to bypass this error."
     ]
   },
+  "INVALID_USE_OF_IDENTIFIER_CLAUSE" : {
+    "message" : [
+      "The use of the IDENTIFIER clause is not allowed for operation: <operation>.",
+      "Please use a regular identifier instead"

Review Comment:
   ```suggestion
         "Please use a regular identifier 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] gengliangwang commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   Thanks for the great work! Merging to master!


-- 
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] srielau commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   > > @gengliangwang @cloud-fan Can you take a peek with your error-context hat on?
   > > As you see my adding withOrigin() into the wrapper has focused many of the contexts around the table names.
   > > Sometimes that is quite right, sometimes that is debatable.
   > 
   > So I am seeing code like
   > 
   > ```
   >         withIdentClause(relation, ident => {
   >           InsertIntoStatement(
   >             UnresolvedRelation(ident),
   >             partition,
   >             cols,
   >             query,
   >             overwrite = false,
   >             ifPartitionNotExists)
   >         })
   > ```
   > 
   > The context `relation` will be applied on the whole `InsertIntoStatement`. I think what you want is on `UnresolvedRelation(ident)` only.
   
   Yes.


-- 
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] gengliangwang commented on pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   Yeah, so we can have code like
   ```
   val unresolvedRelation = withOrigin(ctx) {
     UnresolvedRelation(ident)
   }
    InsertIntoStatement(
               unresolvedRelation,
               partition,
               cols,
               query,
               overwrite = false,
               ifPartitionNotExists)
   ```


-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1492,16 +1520,31 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     } else {
       Seq.empty
     }
-    val name = getFunctionMultiparts(func.functionName)

Review Comment:
   It's not that simple :-(



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -36,6 +36,18 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
 class UnresolvedException(function: String)
   extends AnalysisException(s"Invalid call to $function on unresolved object")
 
+case class PlanWithUnresolvedIdentifier(identifierExpr: Expression,

Review Comment:
   Wouldn't it be nicer to have one way. Are you saying there is no way to handle MERGE using some form of withIdentClause(). What about CREATE TABLE LIKE (that's a DDL command)



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2130,8 +2176,11 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     val filter = Option(ctx.where).map(expression(_))
     val ignoreNulls =
       Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false)
-    val function = UnresolvedFunction(
-      getFunctionMultiparts(ctx.functionName), arguments, isDistinct, filter, ignoreNulls)
+    val function = Option(ctx.functionName.expression()).map(p =>
+      UnresolvedFunctionIdentifierClause(expression(p),

Review Comment:
   So one problem is the identifiers are different. multipartIdentifier has this extra stuff to catch '-' "errorCapturingIdentifier".
   
   I think I'll leave it as is. We re approaching the part where the perfect becomes the enemy of the good.



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3957,51 +4023,66 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
    */
   override def visitDropView(ctx: DropViewContext): AnyRef = withOrigin(ctx) {
     DropView(
-      UnresolvedIdentifier(visitMultipartIdentifier(ctx.multipartIdentifier()), allowTemp = true),
+      withIdentClause(ctx.identifierReference, UnresolvedIdentifier(_, allowTemp = true)),
       ctx.EXISTS != null)
   }
 
   /**
    * Create a [[SetCatalogAndNamespace]] command.
    */
   override def visitUse(ctx: UseContext): LogicalPlan = withOrigin(ctx) {
-    val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier)
-    SetCatalogAndNamespace(UnresolvedNamespace(nameParts))
+      SetCatalogAndNamespace(withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)))

Review Comment:
   ```suggestion
       SetCatalogAndNamespace(withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)))
   ```



-- 
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 #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala:
##########
@@ -276,6 +315,99 @@ object UnresolvedAttribute {
   }
 }
 
+/**
+ * Holds an identifier clause for an attribute that has yet to be resolved.
+ */
+case class UnresolvedAttributeIdentifierClause(expr: Expression)

Review Comment:
   how is this used?



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -272,27 +282,33 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     ctx match {
       case table: InsertIntoTableContext =>
         val (relation, cols, partition, ifPartitionNotExists) = visitInsertIntoTable(table)
-        InsertIntoStatement(
-          relation,
-          partition,
-          cols,
-          query,
-          overwrite = false,
-          ifPartitionNotExists)
+        withIdentClause(relation, ident => {
+          InsertIntoStatement(

Review Comment:
   We only have UnresolvedRelationIdentifierClause because I haven't figured MERGE out.



-- 
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] srielau commented on a diff in pull request #41007: [SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala:
##########
@@ -40,18 +40,34 @@ case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNod
   override def output: Seq[Attribute] = Nil
 }
 
+abstract class UnresolvedTableOrIdentifierClause extends LeafNode
+
 /**
  * Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to
  * [[ResolvedTable]] during analysis.
  */
 case class UnresolvedTable(
     multipartIdentifier: Seq[String],
     commandName: String,
-    relationTypeMismatchHint: Option[String]) extends LeafNode {
+    relationTypeMismatchHint: Option[String]) extends UnresolvedTableOrIdentifierClause {
   override lazy val resolved: Boolean = false
 
   override def output: Seq[Attribute] = Nil
 }
+/**
+ * Holds the identifier of a tabl ein String form that has yet to be looked up in a catalog.

Review Comment:
   Adressed



-- 
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] srielau commented on a diff in pull request #41007: [WIP][SPARK-43205] IDENTIFIER clause

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -2108,6 +2123,18 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
           }
         }
 
+      // Resolve IDENTIFIER clause.
+      case u@UnresolvedRelationIdentifierClause(expr) if expr.resolved =>
+        UnresolvedRelation(IdentifierClauseUtil.evalIdentifierClause(expr))
+
+      // Resolve IDENTIFIER clause and reduce to a regular unresolved table function
+      case u@UnresolvedTableValuedFunctionIdentifierClause(expr, args) if expr.resolved =>
+        UnresolvedTableValuedFunction(IdentifierClauseUtil.evalIdentifierClause(expr), args)
+
+      // Resolve IDENTIFIER clause and reduce to a regular unresolved TVF aliases

Review Comment:
   Addressed



-- 
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 pull request #41007: [SPARK-43205] IDENTIFIER clause

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

   > @dtenedor
   There are still 30 multipartIdentifier usages that do NOT support IDENTIFIER() notation.
   So we would trade mechanical churn in the grammar for code changes in AstBuilder et al.
   
   This is true, we can keep the grammar changes 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