mgaido91 commented on a change in pull request #23531: [SPARK-24497][SQL] 
Support recursive SQL query
URL: https://github.com/apache/spark/pull/23531#discussion_r247440444
 
 

 ##########
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##########
 @@ -205,30 +206,184 @@ class Analyzer(
       CleanupAliases)
   )
 
+  object ResolveRecursiveReferneces extends Rule[LogicalPlan] {
+    def apply(plan: LogicalPlan): LogicalPlan = {
+      val recursiveTables = plan.collect {
+        case rt @ RecursiveTable(name, _, _, _) if rt.anchorResolved => name 
-> rt
+      }.toMap
+
+      plan.resolveOperatorsUp {
+        case UnresolvedRecursiveReference(name) if 
recursiveTables.contains(name) =>
+          RecursiveReference(name, recursiveTables(name).output)
+        case other => other
+      }
+    }
+  }
+
   /**
    * Analyze cte definitions and substitute child plan with analyzed cte 
definitions.
    */
   object CTESubstitution extends Rule[LogicalPlan] {
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
-      case With(child, relations) =>
+      case With(child, relations, allowRecursion) =>
         substituteCTE(child, relations.foldLeft(Seq.empty[(String, 
LogicalPlan)]) {
           case (resolved, (name, relation)) =>
-            resolved :+ name -> executeSameContext(substituteCTE(relation, 
resolved))
-        })
+            val recursiveTableName = if (allowRecursion) Some(name) else None
+            resolved :+
+              name -> executeSameContext(substituteCTE(relation, resolved, 
recursiveTableName))
+        }, None)
       case other => other
     }
 
-    def substituteCTE(plan: LogicalPlan, cteRelations: Seq[(String, 
LogicalPlan)]): LogicalPlan = {
-      plan resolveOperatorsDown {
-        case u: UnresolvedRelation =>
-          cteRelations.find(x => resolver(x._1, u.tableIdentifier.table))
-            .map(_._2).getOrElse(u)
-        case other =>
-          // This cannot be done in ResolveSubquery because ResolveSubquery 
does not know the CTE.
-          other transformExpressions {
-            case e: SubqueryExpression =>
-              e.withNewPlan(substituteCTE(e.plan, cteRelations))
+    def substituteCTE(
+        plan: LogicalPlan,
+        cteRelations: Seq[(String, LogicalPlan)],
+        recursiveTableName: Option[String]): LogicalPlan = {
+      def substitute(
+          plan: LogicalPlan,
+          inSubQuery: Boolean = false): (LogicalPlan, Boolean) = {
+        val references = mutable.Set.empty[UnresolvedRecursiveReference]
+
+        def newReference(recursiveTableName: String) = {
+          val recursiveReference = 
UnresolvedRecursiveReference(recursiveTableName)
+          references += recursiveReference
+
+          recursiveReference
+        }
+
+        val newPlan = plan resolveOperatorsDown {
+          case u: UnresolvedRelation =>
+            val table = u.tableIdentifier.table
+
+            val recursiveReference = recursiveTableName.find(resolver(_, 
table)).map { name =>
+              if (inSubQuery) {
+                throw new AnalysisException(
+                  s"Recursive reference ${name} can't be used in a subquery")
+              }
+
+              newReference(name)
+            }
+
+            recursiveReference
+              .orElse(cteRelations.find(x => resolver(x._1, table)).map(_._2))
+              .getOrElse(u)
+
+          case other =>
+            // This cannot be done in ResolveSubquery because ResolveSubquery 
does not know the CTE.
+            other transformExpressions {
+              case e: SubqueryExpression => e.withNewPlan(substitute(e.plan, 
true)._1)
+            }
+        }
+
+        (newPlan, !references.isEmpty)
+      }
+
+      plan match {
+        case SubqueryAlias(name, u: Union) if recursiveTableName.isDefined =>
+          def combineUnions(union: Union): Seq[LogicalPlan] = 
union.children.flatMap {
+            case u: Union => combineUnions(u)
+            case o => Seq(o)
           }
+
+          val substitutedTerms = combineUnions(u).map(substitute(_))
+          val (anchorTerms, recursiveTerms) = substitutedTerms.partition(!_._2)
+
+          if (!recursiveTerms.isEmpty) {
+            if (anchorTerms.isEmpty) {
+              throw new AnalysisException(s"There should be at least 1 anchor 
term defined in a " +
+                s"recursive query $name")
+            }
+
+            case class PlanTraverseStatus(
+                atRightSideOfLeftOuterJoin: Boolean = false,
+                atLeftSideOfRightOuterJoin: Boolean = false,
+                inFullOuterJoin: Boolean = false,
+                inAggregate: Boolean = false,
+                inDistinct: Boolean = false) {
+              def visitRightSideOfLeftOuterJoin() =
+                if (atRightSideOfLeftOuterJoin) this else 
copy(atRightSideOfLeftOuterJoin = true)
+
+              def visitLeftSideOfRightOuterJoin() =
+                if (atLeftSideOfRightOuterJoin) this else 
copy(atLeftSideOfRightOuterJoin = true)
+
+              def visitFullOuterJoin() = if (inFullOuterJoin) this else 
copy(inFullOuterJoin = true)
+
+              def visitAggregate() = if (inAggregate) this else 
copy(inAggregate = true)
+
+              def visitDistinct() = if (inDistinct) this else copy(inDistinct 
= true)
+            }
+
+            def traversePlanAndCheck(
+                plan: LogicalPlan,
+                status: PlanTraverseStatus = PlanTraverseStatus()): Boolean = 
plan match {
+              case UnresolvedRecursiveReference(name) =>
+                if (status.atRightSideOfLeftOuterJoin) {
+                  throw new AnalysisException(s"Recursive reference ${name} 
can't be used at the " +
+                    s"right side of a left outer join")
+                }
+                if (status.atLeftSideOfRightOuterJoin) {
+                  throw new AnalysisException(s"Recursive reference ${name} 
can't be used at the " +
+                    s"left side of a right outer join")
+                }
+                if (status.inFullOuterJoin) {
+                  throw new AnalysisException(
+                    s"Recursive reference ${name} can't be used in a full 
outer join")
+                }
+                if (status.inAggregate) {
+                  throw new AnalysisException(
+                    s"Recursive reference ${name} can't be used in an 
aggregate")
+                }
+                if (status.inDistinct) {
+                  throw new AnalysisException(
+                    s"Recursive reference ${name} can't be used with distinct")
+                }
+
+                true
+              case Join(left, right, Inner, _, _) =>
+                val l = traversePlanAndCheck(left, status)
+                val r = traversePlanAndCheck(right, status)
+                if (l && r) {
+                  throw new AnalysisException(s"Recursive reference can't be 
used in on both " +
+                    s"side of an inner join")
+                }
+                l || r
+              case Join(left, right, LeftOuter, _, _) =>
+                traversePlanAndCheck(left, status) ||
+                  traversePlanAndCheck(right, 
status.visitRightSideOfLeftOuterJoin())
+              case Join(left, right, RightOuter, _, _) =>
+                traversePlanAndCheck(left, 
status.visitLeftSideOfRightOuterJoin()) ||
+                  traversePlanAndCheck(right, status)
+              case Join(left, right, FullOuter, _, _) =>
+                val newStatus = status.visitFullOuterJoin()
+                traversePlanAndCheck(left, newStatus) || 
traversePlanAndCheck(right, newStatus)
+              case Aggregate(_, _, child) => traversePlanAndCheck(child, 
status.visitAggregate())
+              case Distinct(child) => traversePlanAndCheck(child, 
status.visitDistinct())
+              case o =>
+                o.children.map(traversePlanAndCheck(_, status)).contains(true)
+            }
+
+            val recursiveTermPlans = recursiveTerms.map(_._1)
+
+            recursiveTermPlans.foreach(traversePlanAndCheck(_))
+
+            RecursiveTable(
+              recursiveTableName.get,
+              SubqueryAlias(name, Union(anchorTerms.map(_._1))),
+              Union(recursiveTermPlans),
+              conf.getConf(SQLConf.RECURSION_LEVEL_LIMIT))
+          } else {
+            SubqueryAlias(name, Union(substitutedTerms.map(_._1)))
+          }
+
+        case _ =>
+          val (substitutedPlan, recursiveReferenceFound) = substitute(plan)
+
+          if (recursiveReferenceFound) {
+            throw new AnalysisException(s"Wrong usage of recursive reference " 
+
 
 Review comment:
   nit: remove `s` (please do the same also in the many other cases this happen)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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

Reply via email to