This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new e7ebda6994d3 [SPARK-48273][SQL] Revert "[] Fix late rewrite of 
PlanWithUnresolvedIdentifier
e7ebda6994d3 is described below

commit e7ebda6994d31abd98f3a4863d80d9ed2ba1025b
Author: Wenchen Fan <[email protected]>
AuthorDate: Tue Nov 12 20:18:27 2024 +0800

    [SPARK-48273][SQL] Revert "[] Fix late rewrite of 
PlanWithUnresolvedIdentifier
    
    ### What changes were proposed in this pull request?
    
    This PR reverts https://github.com/apache/spark/pull/46580 (the tests are 
left) because it's no longer needed after 
https://github.com/apache/spark/pull/47501 . The `PlanWithUnresolvedIdentifier` 
becomes more flatten and all its children will be resolved by the early batch 
already.
    
    ### Why are the changes needed?
    
    code cleanup
    
    ### Does this PR introduce _any_ user-facing change?
    
    no
    
    ### How was this patch tested?
    
    existing tests
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    no
    
    Closes #48786 from cloud-fan/ident.
    
    Authored-by: Wenchen Fan <[email protected]>
    Signed-off-by: Wenchen Fan <[email protected]>
---
 .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala     |  9 +++------
 .../spark/sql/catalyst/analysis/ResolveIdentifierClause.scala | 11 +++--------
 .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala     |  4 ++--
 .../org/apache/spark/sql/catalyst/rules/RuleExecutor.scala    |  2 +-
 4 files changed, 9 insertions(+), 17 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e87fe447584a..2e82d7ad39c4 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -268,7 +268,7 @@ class Analyzer(override val catalogManager: CatalogManager) 
extends RuleExecutor
     TypeCoercion.typeCoercionRules
   }
 
-  private def earlyBatches: Seq[Batch] = Seq(
+  override def batches: Seq[Batch] = Seq(
     Batch("Substitution", fixedPoint,
       new SubstituteExecuteImmediate(catalogManager),
       // This rule optimizes `UpdateFields` expression chains so looks more 
like optimization rule.
@@ -289,10 +289,7 @@ class Analyzer(override val catalogManager: 
CatalogManager) extends RuleExecutor
     Batch("Simple Sanity Check", Once,
       LookupFunctions),
     Batch("Keep Legacy Outputs", Once,
-      KeepLegacyOutputs)
-  )
-
-  override def batches: Seq[Batch] = earlyBatches ++ Seq(
+      KeepLegacyOutputs),
     Batch("Resolution", fixedPoint,
       new ResolveCatalogs(catalogManager) ::
       ResolveInsertInto ::
@@ -339,7 +336,7 @@ class Analyzer(override val catalogManager: CatalogManager) 
extends RuleExecutor
       ResolveTimeZone ::
       ResolveRandomSeed ::
       ResolveBinaryArithmetic ::
-      new ResolveIdentifierClause(earlyBatches) ::
+      ResolveIdentifierClause ::
       ResolveUnion ::
       ResolveRowLevelCommandAssignments ::
       MoveParameterizedQueriesDown ::
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala
index 0620f37fa0db..0e1e71a658c8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala
@@ -20,24 +20,19 @@ package org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.catalyst.expressions.{AliasHelper, EvalHelper, 
Expression}
 import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
 import org.apache.spark.sql.catalyst.plans.logical.{CTERelationRef, 
LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.catalyst.trees.TreePattern.{UNRESOLVED_IDENTIFIER, 
UNRESOLVED_IDENTIFIER_WITH_CTE}
 import org.apache.spark.sql.types.StringType
 
 /**
  * Resolves the identifier expressions and builds the original 
plans/expressions.
  */
-class ResolveIdentifierClause(earlyBatches: 
Seq[RuleExecutor[LogicalPlan]#Batch])
-  extends Rule[LogicalPlan] with AliasHelper with EvalHelper {
-
-  private val executor = new RuleExecutor[LogicalPlan] {
-    override def batches: Seq[Batch] = earlyBatches.asInstanceOf[Seq[Batch]]
-  }
+object ResolveIdentifierClause extends Rule[LogicalPlan] with AliasHelper with 
EvalHelper {
 
   override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUpWithPruning(
     _.containsAnyPattern(UNRESOLVED_IDENTIFIER, 
UNRESOLVED_IDENTIFIER_WITH_CTE)) {
     case p: PlanWithUnresolvedIdentifier if p.identifierExpr.resolved && 
p.childrenResolved =>
-      
executor.execute(p.planBuilder.apply(evalIdentifierExpr(p.identifierExpr), 
p.children))
+      p.planBuilder.apply(evalIdentifierExpr(p.identifierExpr), p.children)
     case u @ UnresolvedWithCTERelations(p, cteRelations) =>
       this.apply(p) match {
         case u @ UnresolvedRelation(Seq(table), _, _) =>
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index caeb78d20e6a..044e945d16ad 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -5230,7 +5230,7 @@ class AstBuilder extends DataTypeAstBuilder
     import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
     val query = Option(ctx.query).map(plan)
-    withIdentClause(ctx.identifierReference, ident => {
+    withIdentClause(ctx.identifierReference, query.toSeq, (ident, children) => 
{
       if (query.isDefined && ident.length > 1) {
         val catalogAndNamespace = ident.init
         throw QueryParsingErrors.addCatalogInCacheTableAsSelectNotAllowedError(
@@ -5246,7 +5246,7 @@ class AstBuilder extends DataTypeAstBuilder
         // alongside the text.
         // The same rule can be found in CREATE VIEW builder.
         checkInvalidParameter(query.get, "the query of CACHE TABLE")
-        CacheTableAsSelect(ident.head, query.get, source(ctx.query()), isLazy, 
options)
+        CacheTableAsSelect(ident.head, children.head, source(ctx.query()), 
isLazy, options)
       } else {
         CacheTable(
           createUnresolvedRelation(ctx.identifierReference, ident, None, 
writePrivileges = Nil),
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
index 935233d5c85d..256e1440122d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
@@ -147,7 +147,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] 
extends Logging {
     override val maxIterationsSetting: String = null) extends Strategy
 
   /** A batch of rules. */
-  protected[catalyst] case class Batch(name: String, strategy: Strategy, 
rules: Rule[TreeType]*)
+  protected case class Batch(name: String, strategy: Strategy, rules: 
Rule[TreeType]*)
 
   /** Defines a sequence of rule batches, to be overridden by the 
implementation. */
   protected def batches: Seq[Batch]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to