[SPARK-53127][SQL] Enable LIMIT ALL to override recursion row limit
### What changes were proposed in this pull request?
Introduce LimitAll for LIMIT ALL node that gets pushed into UnionAll to have unbounded number of rows returned by the recursion.
### Why are the changes needed?
LIMIT should override the recursion row limit, so LIMIT ALL should remove this limit. Currently LIMIT ALL is completely no-op (it doesn't create any) node. We introduce this new node and propagate it through its subtree into any UnionLoop.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
New tests in LimitPushdownSuite, golden file test in cte-recursion. Existing golden file tests.
### Was this patch authored or co-authored using generative AI tooling?
No.
Closes #51847 from Pajaraja/pavle-martinovic_data/LimitAllUnionLoop.
Lead-authored-by: pavle-martinovic_data <pavle.martinovic@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
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 df68e7b..cf9b115 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
@@ -407,6 +407,7 @@
       WindowsSubstitution,
       EliminateUnions,
       EliminateLazyExpression),
+    Batch("Apply Limit All", Once, ApplyLimitAll),
     Batch("Disable Hints", Once,
       new ResolveHints.DisableHints),
     Batch("Hints", fixedPoint,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyLimitAll.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyLimitAll.scala
new file mode 100644
index 0000000..51500c9
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyLimitAll.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.plans.logical.{Aggregate, BaseEvalPython, CTERelationRef, Filter, Join, LimitAll, LogicalPlan, Offset, Project, SubqueryAlias, Union, Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+object ApplyLimitAll extends Rule[LogicalPlan] {
+  private def applyLimitAllToPlan(plan: LogicalPlan, isInLimitAll: Boolean = false): LogicalPlan = {
+    plan match {
+      case la: LimitAll =>
+        applyLimitAllToPlan(la.child, isInLimitAll = true)
+      case cteRef: CTERelationRef if isInLimitAll =>
+        cteRef.copy(isUnlimitedRecursion = true)
+      // Allow-list for pushing down Limit All.
+      case _: Project | _: Filter | _: Join | _: Union | _: Offset |
+           _: BaseEvalPython | _: Aggregate | _: Window | _: SubqueryAlias =>
+        plan.withNewChildren(plan.children
+          .map(child => applyLimitAllToPlan(child, isInLimitAll)))
+      case other =>
+        other.withNewChildren(plan.children
+          .map(child => applyLimitAllToPlan(child, isInLimitAll = false)))
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    applyLimitAllToPlan(plan)
+  }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala
index 0f6fcec..5bbb4dde 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala
@@ -286,7 +286,8 @@
         cteDefMap.get(ref.cteId).map { cteDef =>
           // cteDef is certainly resolved, otherwise it would not have been in the map.
           CTERelationRef(
-            cteDef.id, cteDef.resolved, cteDef.output, cteDef.isStreaming, maxRows = cteDef.maxRows)
+            cteDef.id, cteDef.resolved, cteDef.output, cteDef.isStreaming, maxRows = cteDef.maxRows,
+              isUnlimitedRecursion = ref.isUnlimitedRecursion)
         }.getOrElse {
           ref
         }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index b416882..1d7cf54 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -405,6 +405,8 @@
 
       def globalLimit(limitExpr: Expression): LogicalPlan = GlobalLimit(limitExpr, logicalPlan)
 
+      def limitAll(): LogicalPlan = LimitAll(logicalPlan)
+
       def offset(offsetExpr: Expression): LogicalPlan = Offset(offsetExpr, logicalPlan)
 
       def join(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala
index ad1a1a9..44b31d3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala
@@ -22,7 +22,7 @@
 import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations
 import org.apache.spark.sql.catalyst.expressions.{Alias, OuterReference, SubqueryExpression}
 import org.apache.spark.sql.catalyst.plans.Inner
-import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Join, JoinHint, LogicalPlan, Project, Subquery, WithCTE}
+import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Join, JoinHint, LogicalPlan, Project, Subquery, UnionLoop, WithCTE}
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION}
 
@@ -189,14 +189,20 @@
 
       case ref: CTERelationRef =>
         val refInfo = cteMap(ref.cteId)
+
+        val cteBody = if (ref.isUnlimitedRecursion) {
+          setUnlimitedRecursion(refInfo.cteDef.child, ref.cteId)
+        } else {
+          refInfo.cteDef.child
+        }
         if (refInfo.shouldInline) {
           if (ref.outputSet == refInfo.cteDef.outputSet) {
-            refInfo.cteDef.child
+            cteBody
           } else {
             val ctePlan = DeduplicateRelations(
               Join(
-                refInfo.cteDef.child,
-                refInfo.cteDef.child,
+                cteBody,
+                cteBody,
                 Inner,
                 None,
                 JoinHint(None, None)
@@ -226,6 +232,18 @@
       case _ => plan
     }
   }
+
+  // Helper function to set unlimited recursion.
+  private def setUnlimitedRecursion(plan: LogicalPlan, id: Long): LogicalPlan = {
+    plan match {
+      case ul: UnionLoop if ul.id == id =>
+        // Since there is exactly one UnionLoop node with this id in the CTE body, we can stop the
+        // recursion here.
+        ul.copy(limit = Some(-1))
+      case other =>
+        other.withNewChildren(plan.children.map(child => setUnlimitedRecursion(child, id)))
+    }
+  }
 }
 
 /**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala
index 32c2b56..421c47f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala
@@ -141,7 +141,7 @@
         cteDef
       }
 
-    case cteRef @ CTERelationRef(cteId, _, output, _, _, _, _) =>
+    case cteRef @ CTERelationRef(cteId, _, output, _, _, _, _, _) =>
       val (cteDef, _, _, newAttrSet) = cteMap(cteId)
       if (needsPruning(cteDef.child, newAttrSet)) {
         val indices = newAttrSet.toSeq.map(cteDef.output.indexOf)
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 dd768b7..e43e32f 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
@@ -1345,14 +1345,20 @@
     }
 
     // LIMIT
-    // - LIMIT ALL is the same as omitting the LIMIT clause
-    withOffset.optional(limit) {
-      if (forPipeOperators && clause.nonEmpty && clause != PipeOperators.offsetClause) {
-        throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError(
-          ctx, clause, PipeOperators.limitClause)
-      }
+    if (forPipeOperators && clause.nonEmpty
+      && clause != PipeOperators.offsetClause && limit != null) {
+      throw QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError(
+        ctx, clause, PipeOperators.limitClause)
+    }
+    // LIMIT ALL creates LimitAll node which can be used for infinite recursions in recursive CTEs.
+    if (ctx.ALL() != null) {
       clause = PipeOperators.limitClause
-      Limit(typedVisit(limit), withOffset)
+      LimitAll(withOffset)
+    } else {
+      withOffset.optional(limit) {
+        clause = PipeOperators.limitClause
+        Limit(typedVisit(limit), withOffset)
+      }
     }
   }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index 810f2b0..ad69394 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -1659,6 +1659,22 @@
     copy(child = newChild)
 }
 
+/**
+ * Logical node that represents the LIMIT ALL operation. This operation is usually no-op and exists
+ * to provide compatability with other databases. However, in case of recursive CTEs, Limit nodes
+ * serve another purpose, to override the default row limit which is determined by a flag. As a
+ * result, LIMIT ALL should also be used to completely negate the row limit, which is exactly what
+ * this node is used for.
+ */
+case class LimitAll(child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output
+
+  final override val nodePatterns: Seq[TreePattern] = Seq(LIMIT)
+
+  override protected def withNewChildInternal(newChild: LogicalPlan): LimitAll =
+    copy(child = newChild)
+}
+
 object OffsetAndLimit {
   def unapply(p: GlobalLimit): Option[(Int, Int, LogicalPlan)] = {
     p match {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala
index c3c662c..373f8b3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/cteOperators.scala
@@ -128,7 +128,7 @@
   override def output: Seq[Attribute] = if (resolved) child.output else Nil
 
   lazy val hasSelfReferenceAsCTERef: Boolean = child.collectFirstWithSubqueries {
-    case CTERelationRef(this.id, _, _, _, _, true, _) => true
+    case CTERelationRef(this.id, _, _, _, _, true, _, _) => true
   }.getOrElse(false)
   lazy val hasSelfReferenceInAnchor: Boolean = {
     val unionNode: Option[Union] = child match {
@@ -144,7 +144,7 @@
     }
     if (unionNode.isDefined) {
       unionNode.get.children.head.collectFirstWithSubqueries {
-        case CTERelationRef(this.id, _, _, _, _, true, _) => true
+        case CTERelationRef(this.id, _, _, _, _, true, _, _) => true
       }.getOrElse(false)
     } else {
       false
@@ -160,7 +160,7 @@
     }
     if (withCTENode.isDefined) {
       withCTENode.exists(_.cteDefs.exists(_.collectFirstWithSubqueries {
-        case CTERelationRef(this.id, _, _, _, _, true, _) => true
+        case CTERelationRef(this.id, _, _, _, _, true, _, _) => true
       }.isDefined))
     } else {
       false
@@ -194,7 +194,8 @@
     override val isStreaming: Boolean,
     statsOpt: Option[Statistics] = None,
     recursive: Boolean = false,
-    override val maxRows: Option[Long] = None) extends LeafNode with MultiInstanceRelation {
+    override val maxRows: Option[Long] = None,
+    isUnlimitedRecursion: Boolean = false) extends LeafNode with MultiInstanceRelation {
 
   final override val nodePatterns: Seq[TreePattern] = Seq(CTE)
 
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala
index 47b5ba3..c6d4091 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala
@@ -106,7 +106,7 @@
   private def executeAndCacheAndCount(plan: LogicalPlan, currentLimit: Int) = {
     // In case limit is defined, we create a (local) limit node above the plan and execute
     // the newly created plan.
-    val planWithLimit = if (limit.isDefined) {
+    val planWithLimit = if (limit.isDefined && limit.get >= 0) {
       LocalLimit(Literal(currentLimit), plan)
     } else {
       plan
@@ -167,6 +167,8 @@
     // the user knows they aren't getting all the rows they requested.
     var currentLimit = limit.getOrElse(rowLimit)
 
+    val unlimitedRecursion = currentLimit == -1
+
     val userSpecifiedLimit = limit.isDefined
 
     val unionChildren = mutable.ArrayBuffer.empty[LogicalPlan]
@@ -240,7 +242,7 @@
 
       unionChildren += prevPlan
 
-      if (rowLimit != -1) {
+      if (!unlimitedRecursion) {
         currentLimit -= prevCount.toInt
         if (currentLimit <= 0) {
           if (userSpecifiedLimit) {
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out
index 454682e..b10cf52 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out
@@ -183,7 +183,7 @@
 :           +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
 +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
    +- SubqueryAlias cte
-      +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false
+      +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false, false
 
 
 -- !query
@@ -232,7 +232,7 @@
 :        +- LocalLimit 1
 :           +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
 :              +- SubqueryAlias cte
-:                 +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false
+:                 +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false, false
 +- OneRowRelation
 
 
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out
index 5ca0f8a..bb0fef4 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out
@@ -10,7 +10,7 @@
       :        +- OneRowRelation
       +- Project [col#x]
          +- SubqueryAlias s
-            +- CTERelationRef xxxx, true, [col#x], false, false, 1
+            +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -32,7 +32,7 @@
       :        +- OneRowRelation
       +- Project [col#x]
          +- SubqueryAlias s
-            +- CTERelationRef xxxx, true, [col#x], false, false, 1
+            +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -49,7 +49,7 @@
             :        +- OneRowRelation
             +- Project [col#x]
                +- SubqueryAlias s
-                  +- CTERelationRef xxxx, true, [col#x], false, false, 1
+                  +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -64,7 +64,7 @@
    :        +- OneRowRelation
    +- Project [col#x]
       +- SubqueryAlias S
-         +- CTERelationRef xxxx, true, [col#x], false, false, 1
+         +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -86,7 +86,7 @@
    :        +- OneRowRelation
    +- Project [col#x]
       +- SubqueryAlias s
-         +- CTERelationRef xxxx, true, [col#x], false, false, 1
+         +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out
index 3c48bd4..43bc4bb 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out
@@ -15,10 +15,10 @@
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -37,7 +37,7 @@
       :           +- OneRowRelation
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x], false, false, 1
+            +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -54,7 +54,7 @@
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- OneRowRelation
 
 
@@ -140,10 +140,10 @@
 :  +- SubqueryAlias t2
 :     +- Project [2#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [2#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
 +- Project [2#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [2#x], false, false, 1
+      +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
 
 
 -- !query
@@ -178,11 +178,11 @@
 :        :           :           +- OneRowRelation
 :        :           +- Project [c#x]
 :        :              +- SubqueryAlias t
-:        :                 +- CTERelationRef xxxx, true, [c#x], false, false, 1
+:        :                 +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 :        +- OneRowRelation
 +- Project [scalarsubquery()#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1
+      +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1, false
 
 
 -- !query
@@ -215,15 +215,15 @@
 :  +- SubqueryAlias t2
 :     +- Project [3#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [3#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [3#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [3#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [3#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [3#x], false, false, 1, false
 +- Project [3#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [3#x], false, false, 1
+      +- CTERelationRef xxxx, true, [3#x], false, false, 1, false
 
 
 -- !query
@@ -248,7 +248,7 @@
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x], false, false, 1
+            +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -277,7 +277,7 @@
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x], false, false, 1
+                  +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -312,7 +312,7 @@
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x], false, false, 1
+                  +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -335,7 +335,7 @@
    :     :        +- OneRowRelation
    :     +- Project [2#x]
    :        +- SubqueryAlias t
-   :           +- CTERelationRef xxxx, true, [2#x], false, false, 1
+   :           +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
    +- OneRowRelation
 
 
@@ -362,7 +362,7 @@
    :     :     :        +- OneRowRelation
    :     :     +- Project [2#x]
    :     :        +- SubqueryAlias t
-   :     :           +- CTERelationRef xxxx, true, [2#x], false, false, 1
+   :     :           +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
    :     +- OneRowRelation
    +- OneRowRelation
 
@@ -396,7 +396,7 @@
    :        :     :        +- OneRowRelation
    :        :     +- Project [3#x]
    :        :        +- SubqueryAlias t
-   :        :           +- CTERelationRef xxxx, true, [3#x], false, false, 1
+   :        :           +- CTERelationRef xxxx, true, [3#x], false, false, 1, false
    :        +- OneRowRelation
    +- OneRowRelation
 
@@ -425,9 +425,9 @@
       :     :           +- OneRowRelation
       :     +- Project [c#x]
       :        +- SubqueryAlias t
-      :           +- CTERelationRef xxxx, true, [c#x], false, false, 1
+      :           +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
       +- SubqueryAlias t
-         +- CTERelationRef xxxx, true, [c#x], false, false, 1
+         +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -448,14 +448,14 @@
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [2 AS 2#x]
 :        +- OneRowRelation
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -480,10 +480,10 @@
 :  +- SubqueryAlias t
 :     +- Project [2#x]
 :        +- SubqueryAlias aBC
-:           +- CTERelationRef xxxx, true, [2#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
 +- Project [2#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [2#x], false, false, 1
+      +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
 
 
 -- !query
@@ -506,7 +506,7 @@
    :     :        +- OneRowRelation
    :     +- Project [2#x]
    :        +- SubqueryAlias aBC
-   :           +- CTERelationRef xxxx, true, [2#x], false, false, 1
+   :           +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
    +- OneRowRelation
 
 
@@ -530,15 +530,15 @@
 :  +- SubqueryAlias t3
 :     +- Project [1#x]
 :        +- SubqueryAlias t1
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [1#x]
 :        +- SubqueryAlias t3
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -561,12 +561,12 @@
 :  +- SubqueryAlias cte_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x], false, false, 1
+            +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -594,19 +594,19 @@
 :  +- SubqueryAlias cte_inner_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias cte_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias __auto_generated_subquery_name
 :           +- Project [1#x]
 :              +- SubqueryAlias cte_inner_inner
-:                 +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:                 +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x], false, false, 1
+            +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out
index 70d00a1..0e831ef 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out
@@ -15,10 +15,10 @@
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -37,7 +37,7 @@
       :           +- OneRowRelation
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x], false, false, 1
+            +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -54,7 +54,7 @@
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- OneRowRelation
 
 
@@ -171,11 +171,11 @@
 :        :           :           +- OneRowRelation
 :        :           +- Project [c#x]
 :        :              +- SubqueryAlias t
-:        :                 +- CTERelationRef xxxx, true, [c#x], false, false, 1
+:        :                 +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 :        +- OneRowRelation
 +- Project [scalarsubquery()#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1
+      +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false, 1, false
 
 
 -- !query
@@ -225,7 +225,7 @@
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x], false, false, 1
+            +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -254,7 +254,7 @@
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x], false, false, 1
+                  +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -289,7 +289,7 @@
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x], false, false, 1
+                  +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
 
 
 -- !query
@@ -392,14 +392,14 @@
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [2 AS 2#x]
 :        +- OneRowRelation
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -462,15 +462,15 @@
 :  +- SubqueryAlias t3
 :     +- Project [1#x]
 :        +- SubqueryAlias t1
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [1#x]
 :        +- SubqueryAlias t3
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -493,12 +493,12 @@
 :  +- SubqueryAlias cte_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x], false, false, 1
+            +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
@@ -526,19 +526,19 @@
 :  +- SubqueryAlias cte_inner_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias cte_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias __auto_generated_subquery_name
 :           +- Project [1#x]
 :              +- SubqueryAlias cte_inner_inner
-:                 +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:                 +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x], false, false, 1
+            +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out
index 23ee616..848cdce 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out
@@ -44,7 +44,7 @@
 :                 +- UnionLoopRef xxxx, [level#x], false
 +- Project [level#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -68,7 +68,7 @@
 :                       +- UnionLoopRef xxxx, [col1#x], false
 +- Project [level#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -93,7 +93,7 @@
 :                       +- UnionLoopRef xxxx, [a#x], false
 +- Project [c#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [c#x], false, false
+      +- CTERelationRef xxxx, true, [c#x], false, false, false
 
 
 -- !query
@@ -116,7 +116,7 @@
 :                    +- UnionLoopRef xxxx, [col1#x], false
 +- Project [level#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -160,7 +160,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [n#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -190,7 +190,7 @@
 :                       +- UnionLoopRef xxxx, [col1#x], false
 +- Project [level#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -238,7 +238,7 @@
 :                       +- UnionLoopRef xxxx, [col1#x], false
 +- Project [level#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -284,7 +284,7 @@
    +- LocalLimit 60
       +- Project [n#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -294,6 +294,108 @@
 
 
 -- !query
+WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS (
+    SELECT 1
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+SELECT * FROM t LIMIT ALL
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false, 100
+:  +- SubqueryAlias t
+:     +- Project [1#x AS n#x]
+:        +- UnionLoop xxxx, 100
+:           :- Project [1 AS 1#x]
+:           :  +- OneRowRelation
+:           +- Project [(n#x + 1) AS (n + 1)#x]
+:              +- Filter (n#x < 60)
+:                 +- SubqueryAlias t
+:                    +- Project [1#x AS n#x]
+:                       +- UnionLoopRef xxxx, [1#x], false
++- Project [n#x]
+   +- SubqueryAlias t
+      +- CTERelationRef xxxx, true, [n#x], false, false, true
+
+
+-- !query
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+SELECT * FROM t LIMIT ALL
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false, 100
+:  +- SubqueryAlias t
+:     +- UnionLoop xxxx, 100
+:        :- Project [1 AS n#x]
+:        :  +- OneRowRelation
+:        +- Project [(n#x + 1) AS (n + 1)#x]
+:           +- Filter (n#x < 60)
+:              +- SubqueryAlias t
+:                 +- UnionLoopRef xxxx, [n#x], false
++- Project [n#x]
+   +- SubqueryAlias t
+      +- CTERelationRef xxxx, true, [n#x], false, false, true
+
+
+-- !query
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+   (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t)
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false, 100
+:  +- SubqueryAlias t
+:     +- UnionLoop xxxx, 100
+:        :- Project [1 AS n#x]
+:        :  +- OneRowRelation
+:        +- Project [(n#x + 1) AS (n + 1)#x]
+:           +- Filter (n#x < 60)
+:              +- SubqueryAlias t
+:                 +- UnionLoopRef xxxx, [n#x], false
++- Union false, false
+   :- Project [n#x]
+   :  +- SubqueryAlias t
+   :     +- CTERelationRef xxxx, true, [n#x], false, false, true
+   +- Project [n#x]
+      +- SubqueryAlias t
+         +- CTERelationRef xxxx, true, [n#x], false, false, false
+
+
+-- !query
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+   (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t LIMIT ALL)
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false, 100
+:  +- SubqueryAlias t
+:     +- UnionLoop xxxx, 100
+:        :- Project [1 AS n#x]
+:        :  +- OneRowRelation
+:        +- Project [(n#x + 1) AS (n + 1)#x]
+:           +- Filter (n#x < 60)
+:              +- SubqueryAlias t
+:                 +- UnionLoopRef xxxx, [n#x], false
++- Union false, false
+   :- Project [n#x]
+   :  +- SubqueryAlias t
+   :     +- CTERelationRef xxxx, true, [n#x], false, false, true
+   +- Project [n#x]
+      +- SubqueryAlias t
+         +- CTERelationRef xxxx, true, [n#x], false, false, true
+
+
+-- !query
 WITH RECURSIVE r(level) AS (
   VALUES 0
   UNION ALL
@@ -315,7 +417,7 @@
    +- LocalLimit 10
       +- Project [level#x]
          +- SubqueryAlias r
-            +- CTERelationRef xxxx, true, [level#x], false, false
+            +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -385,7 +487,7 @@
       +- Sort [level#x ASC NULLS FIRST], true
          +- Project [level#x, level#x]
             +- SubqueryAlias r
-               +- CTERelationRef xxxx, true, [level#x], false, false
+               +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -435,7 +537,7 @@
 :              +- OneRowRelation
 +- Project [n#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [n#x], false, false, 2
+      +- CTERelationRef xxxx, true, [n#x], false, false, 2, false
 
 
 -- !query
@@ -514,7 +616,7 @@
    +- LocalLimit 5
       +- Project [n#x]
          +- SubqueryAlias t1
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -543,10 +645,10 @@
 :           :              +- UnionLoopRef xxxx, [level#x], false
 :           +- Project [(level + 1)#x]
 :              +- SubqueryAlias t2
-:                 +- CTERelationRef xxxx, true, [(level + 1)#x], false, false
+:                 +- CTERelationRef xxxx, true, [(level + 1)#x], false, false, false
 +- Project [level#x]
    +- SubqueryAlias t1
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -590,15 +692,15 @@
 :           :              +- UnionLoopRef xxxx, [level#x], false
 :           +- Project [(level + 1)#x]
 :              +- SubqueryAlias t3
-:                 +- CTERelationRef xxxx, true, [(level + 1)#x], false, false
+:                 +- CTERelationRef xxxx, true, [(level + 1)#x], false, false, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [level#x]
 :        +- SubqueryAlias t1
-:           +- CTERelationRef xxxx, true, [level#x], false, false
+:           +- CTERelationRef xxxx, true, [level#x], false, false, false
 +- Project [level#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -637,7 +739,7 @@
 :                             +- UnionLoopRef xxxx, [col1#x, col2#x], false
 +- Project [level#x, data#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x, data#x], false, false
+      +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false
 
 
 -- !query
@@ -670,7 +772,7 @@
 :                          +- OneRowRelation
 +- Project [level#x, data#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x, data#x], false, false
+      +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false
 
 
 -- !query
@@ -723,7 +825,7 @@
 :                          +- UnionLoopRef xxxx, [col1#x, col2#x], false
 +- Project [level#x, data#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x, data#x], false, false
+      +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false
 
 
 -- !query
@@ -796,7 +898,7 @@
 :                          +- OneRowRelation
 +- Project [level#x, data#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x, data#x], false, false
+      +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false
 
 
 -- !query
@@ -848,7 +950,7 @@
 :                          +- OneRowRelation
 +- Project [level#x, data#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x, data#x], false, false
+      +- CTERelationRef xxxx, true, [level#x, data#x], false, false, false
 
 
 -- !query
@@ -927,7 +1029,7 @@
       :                       +- UnionLoopRef xxxx, [col1#x], false
       +- Project [level#x]
          +- SubqueryAlias r
-            +- CTERelationRef xxxx, true, [level#x], false, false
+            +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -950,7 +1052,7 @@
             :                       +- UnionLoopRef xxxx, [col1#x], false
             +- Project [level#x]
                +- SubqueryAlias r
-                  +- CTERelationRef xxxx, true, [level#x], false, false
+                  +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -987,7 +1089,7 @@
    :                       +- UnionLoopRef xxxx, [col1#x], false
    +- Project [level#x]
       +- SubqueryAlias r
-         +- CTERelationRef xxxx, true, [level#x], false, false
+         +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -1036,11 +1138,11 @@
    :- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/rt2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/rt2], Append, `spark_catalog`.`default`.`rt2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/rt2), [level]
    :  +- Project [level#x]
    :     +- SubqueryAlias r
-   :        +- CTERelationRef xxxx, true, [level#x], false, false
+   :        +- CTERelationRef xxxx, true, [level#x], false, false, false
    +- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/rt2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/rt2], Append, `spark_catalog`.`default`.`rt2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/rt2), [level]
       +- Project [level#x]
          +- SubqueryAlias r
-            +- CTERelationRef xxxx, true, [level#x], false, false
+            +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -1096,9 +1198,9 @@
 +- Project [level#x, level#x]
    +- Join Inner, (level#x = (level#x + 10))
       :- SubqueryAlias r1
-      :  +- CTERelationRef xxxx, true, [level#x], false, false
+      :  +- CTERelationRef xxxx, true, [level#x], false, false, false
       +- SubqueryAlias r2
-         +- CTERelationRef xxxx, true, [level#x], false, false
+         +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -1125,10 +1227,10 @@
    +- Join Inner, (level#x = level#x)
       :- SubqueryAlias r1
       :  +- SubqueryAlias r
-      :     +- CTERelationRef xxxx, true, [level#x], false, false
+      :     +- CTERelationRef xxxx, true, [level#x], false, false, false
       +- SubqueryAlias r2
          +- SubqueryAlias r
-            +- CTERelationRef xxxx, true, [level#x], false, false
+            +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -1160,14 +1262,14 @@
 :        +- UnionLoop xxxx
 :           :- Project [0 AS outerlevel#x, innerlevel#x]
 :           :  +- SubqueryAlias r1
-:           :     +- CTERelationRef xxxx, true, [innerlevel#x], false, false
+:           :     +- CTERelationRef xxxx, true, [innerlevel#x], false, false, false
 :           +- Project [(outerlevel#x + 1) AS (outerlevel + 1)#x, innerlevel#x]
 :              +- Filter (outerlevel#x < 3)
 :                 +- SubqueryAlias r2
 :                    +- UnionLoopRef xxxx, [outerlevel#x, innerlevel#x], false
 +- Project [outerlevel#x, innerlevel#x]
    +- SubqueryAlias r2
-      +- CTERelationRef xxxx, true, [outerlevel#x, innerlevel#x], false, false
+      +- CTERelationRef xxxx, true, [outerlevel#x, innerlevel#x], false, false, false
 
 
 -- !query
@@ -1201,14 +1303,14 @@
 :           +- Union false, false
 :              :- Project [level#x]
 :              :  +- SubqueryAlias r
-:              :     +- CTERelationRef xxxx, true, [level#x], false, false
+:              :     +- CTERelationRef xxxx, true, [level#x], false, false, false
 :              +- Project [(level#x + 1) AS (level + 1)#x]
 :                 +- Filter (level#x < 3)
 :                    +- SubqueryAlias r
-:                       +- CTERelationRef xxxx, true, [level#x], false, false
+:                       +- CTERelationRef xxxx, true, [level#x], false, false, false
 +- Project [level#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -1242,7 +1344,7 @@
 :           :  :                       +- UnionLoopRef xxxx, [col1#x], false
 :           :  +- Project [level#x]
 :           :     +- SubqueryAlias r
-:           :        +- CTERelationRef xxxx, true, [level#x], false, false
+:           :        +- CTERelationRef xxxx, true, [level#x], false, false, false
 :           +- Project [(level#x + 1) AS (level + 1)#x]
 :              +- Filter (level#x < 3)
 :                 +- SubqueryAlias r
@@ -1250,7 +1352,7 @@
 :                       +- UnionLoopRef xxxx, [level#x], false
 +- Project [level#x]
    +- SubqueryAlias r
-      +- CTERelationRef xxxx, true, [level#x], false, false
+      +- CTERelationRef xxxx, true, [level#x], false, false, false
 
 
 -- !query
@@ -1298,7 +1400,7 @@
 :                    +- UnionLoopRef xxxx, [destination#x, path#x, length#x], false
 +- Project [destination#x, path#x, length#x]
    +- SubqueryAlias destinations_from_new_york
-      +- CTERelationRef xxxx, true, [destination#x, path#x, length#x], false, false
+      +- CTERelationRef xxxx, true, [destination#x, path#x, length#x], false, false, false
 
 
 -- !query
@@ -1328,7 +1430,7 @@
 +- Sort [a#x ASC NULLS FIRST], true
    +- Project [a#x]
       +- SubqueryAlias fibonacci
-         +- CTERelationRef xxxx, true, [a#x, b#x], false, false
+         +- CTERelationRef xxxx, true, [a#x, b#x], false, false, false
 
 
 -- !query
@@ -1353,7 +1455,7 @@
    +- LocalLimit 5
       +- Project [a#x]
          +- SubqueryAlias t1
-            +- CTERelationRef xxxx, true, [a#x, b#x, c#x], false, false
+            +- CTERelationRef xxxx, true, [a#x, b#x, c#x], false, false, false
 
 
 -- !query
@@ -1396,7 +1498,7 @@
    +- LocalLimit 63
       +- Project [n#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -1439,10 +1541,10 @@
 :                    :  +- Project [id#x AS id#x, xid#x AS xid#x]
 :                    :     +- UnionLoopRef xxxx, [id#x, xid#x], false
 :                    +- SubqueryAlias x
-:                       +- CTERelationRef xxxx, true, [id#x], false, false, 2
+:                       +- CTERelationRef xxxx, true, [id#x], false, false, 2, false
 +- Project [id#x, xid#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [id#x, xid#x], false, false
+      +- CTERelationRef xxxx, true, [id#x, xid#x], false, false, false
 
 
 -- !query
@@ -1483,10 +1585,10 @@
 :                    :  +- Project [1#x AS n#x]
 :                    :     +- UnionLoopRef xxxx, [1#x], false
 :                    +- SubqueryAlias t1
-:                       +- CTERelationRef xxxx, true, [a#x, b#x], false, false
+:                       +- CTERelationRef xxxx, true, [a#x, b#x], false, false, false
 +- Project [n#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -1524,14 +1626,14 @@
 :              :     +- Project [j#x AS k#x]
 :              :        +- Project [j#x]
 :              :           +- SubqueryAlias t2
-:              :              +- CTERelationRef xxxx, true, [j#x], false, false
+:              :              +- CTERelationRef xxxx, true, [j#x], false, false, false
 :              +- Project [k#x]
 :                 +- Filter (k#x <= 5)
 :                    +- SubqueryAlias t3
-:                       +- CTERelationRef xxxx, true, [k#x], false, false
+:                       +- CTERelationRef xxxx, true, [k#x], false, false, false
 +- Project [n#x]
    +- SubqueryAlias t1
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -1564,7 +1666,7 @@
 :           +- UnionLoop xxxx
 :              :- Project [0 AS outerlevel#x, innerlevel#x]
 :              :  +- SubqueryAlias r1
-:              :     +- CTERelationRef xxxx, true, [innerlevel#x], false, false
+:              :     +- CTERelationRef xxxx, true, [innerlevel#x], false, false, false
 :              +- Project [(outerlevel1#x + 1) AS (outerlevel1 + 1)#x, innerlevel1#x]
 :                 +- Filter (outerlevel1#x < 3)
 :                    +- SubqueryAlias r2
@@ -1572,7 +1674,7 @@
 :                          +- UnionLoopRef xxxx, [outerlevel#x, innerlevel#x], false
 +- Project [outerlevel1#x, innerlevel1#x]
    +- SubqueryAlias r2
-      +- CTERelationRef xxxx, true, [outerlevel1#x, innerlevel1#x], false, false
+      +- CTERelationRef xxxx, true, [outerlevel1#x, innerlevel1#x], false, false, false
 
 
 -- !query
@@ -1627,7 +1729,7 @@
 :           :- Project [x#x, x#x]
 :           :  +- Filter (x#x = 1)
 :           :     +- SubqueryAlias tmp
-:           :        +- CTERelationRef xxxx, true, [x#x], false, false, 5
+:           :        +- CTERelationRef xxxx, true, [x#x], false, false, 5, false
 :           +- Project [(x#x + 1) AS (x + 1)#x, x#x]
 :              +- Filter (x#x < 5)
 :                 +- SubqueryAlias rcte
@@ -1635,7 +1737,7 @@
 :                       +- UnionLoopRef xxxx, [x#x, x#x], false
 +- Project [x#x, y#x]
    +- SubqueryAlias rcte
-      +- CTERelationRef xxxx, true, [x#x, y#x], false, false
+      +- CTERelationRef xxxx, true, [x#x, y#x], false, false, false
 
 
 -- !query
@@ -1666,7 +1768,7 @@
 :           :- Project [x#x, x#x]
 :           :  +- Filter (x#x = 1)
 :           :     +- SubqueryAlias tmp
-:           :        +- CTERelationRef xxxx, true, [x#x], false, false, 5
+:           :        +- CTERelationRef xxxx, true, [x#x], false, false, 5, false
 :           +- Project [(x#x + 1) AS (x + 1)#x, x#x]
 :              +- Filter (x#x < 5)
 :                 +- SubqueryAlias rcte
@@ -1674,7 +1776,7 @@
 :                       +- UnionLoopRef xxxx, [x#x, x#x], false
 +- Project [x#x, y#x]
    +- SubqueryAlias rcte
-      +- CTERelationRef xxxx, true, [x#x, y#x], false, false
+      +- CTERelationRef xxxx, true, [x#x, y#x], false, false, false
 
 
 -- !query
@@ -1705,7 +1807,7 @@
 :           :- Project [x#x, x#x, x#x, x#x]
 :           :  +- Filter (x#x = 1)
 :           :     +- SubqueryAlias tmp
-:           :        +- CTERelationRef xxxx, true, [x#x], false, false, 5
+:           :        +- CTERelationRef xxxx, true, [x#x], false, false, 5, false
 :           +- Project [(x#x + 1) AS (x + 1)#x, x#x, (y#x + 1) AS (y + 1)#x, y#x]
 :              +- Filter (x#x < 5)
 :                 +- SubqueryAlias rcte
@@ -1713,7 +1815,7 @@
 :                       +- UnionLoopRef xxxx, [x#x, x#x, x#x, x#x], false
 +- Project [x#x, y#x, z#x, t#x]
    +- SubqueryAlias rcte
-      +- CTERelationRef xxxx, true, [x#x, y#x, z#x, t#x], false, false
+      +- CTERelationRef xxxx, true, [x#x, y#x, z#x, t#x], false, false, false
 
 
 -- !query
@@ -1810,7 +1912,7 @@
 :                          +- UnionLoopRef xxxx, [1#x, CAST(1 AS BIGINT)#xL], false
 +- Project [n#x, m#xL]
    +- SubqueryAlias t1
-      +- CTERelationRef xxxx, true, [n#x, m#xL], false, false
+      +- CTERelationRef xxxx, true, [n#x, m#xL], false, false, false
 
 
 -- !query
@@ -1854,7 +1956,7 @@
    +- LocalLimit 25
       +- Project [n#x]
          +- SubqueryAlias t1
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -1882,14 +1984,14 @@
    :  :  +- SubqueryAlias __auto_generated_subquery_name
    :  :     +- Project [n#x]
    :  :        +- SubqueryAlias t1
-   :  :           +- CTERelationRef xxxx, true, [n#x], false, false
+   :  :           +- CTERelationRef xxxx, true, [n#x], false, false, false
    :  +- Aggregate [sum(n#x) AS sum(n)#xL]
    :     +- SubqueryAlias __auto_generated_subquery_name
    :        +- GlobalLimit 3
    :           +- LocalLimit 3
    :              +- Project [n#x]
    :                 +- SubqueryAlias t1
-   :                    +- CTERelationRef xxxx, true, [n#x], false, false
+   :                    +- CTERelationRef xxxx, true, [n#x], false, false, false
    +- OneRowRelation
 
 
@@ -1919,14 +2021,14 @@
    :  :        +- LocalLimit 5
    :  :           +- Project [n#x]
    :  :              +- SubqueryAlias t1
-   :  :                 +- CTERelationRef xxxx, true, [n#x], false, false
+   :  :                 +- CTERelationRef xxxx, true, [n#x], false, false, false
    :  +- Aggregate [sum(n#x) AS sum(n)#xL]
    :     +- SubqueryAlias __auto_generated_subquery_name
    :        +- GlobalLimit 3
    :           +- LocalLimit 3
    :              +- Project [n#x]
    :                 +- SubqueryAlias t1
-   :                    +- CTERelationRef xxxx, true, [n#x], false, false
+   :                    +- CTERelationRef xxxx, true, [n#x], false, false, false
    +- OneRowRelation
 
 
@@ -1964,7 +2066,7 @@
 :           :  :           +- LocalLimit 10
 :           :  :              +- Project [n#x]
 :           :  :                 +- SubqueryAlias t1
-:           :  :                    +- CTERelationRef xxxx, true, [n#x], false, false
+:           :  :                    +- CTERelationRef xxxx, true, [n#x], false, false, false
 :           :  +- OneRowRelation
 :           +- Project [(m#xL + scalar-subquery#x []) AS (m + scalarsubquery())#xL]
 :              :  +- Aggregate [sum(n#x) AS sum(n)#xL]
@@ -1973,7 +2075,7 @@
 :              :           +- LocalLimit 3
 :              :              +- Project [n#x]
 :              :                 +- SubqueryAlias t1
-:              :                    +- CTERelationRef xxxx, true, [n#x], false, false
+:              :                    +- CTERelationRef xxxx, true, [n#x], false, false, false
 :              +- SubqueryAlias t2
 :                 +- Project [scalarsubquery()#xL AS m#xL]
 :                    +- UnionLoopRef xxxx, [scalarsubquery()#xL], false
@@ -1981,7 +2083,7 @@
    +- LocalLimit 20
       +- Project [m#xL]
          +- SubqueryAlias t2
-            +- CTERelationRef xxxx, true, [m#xL], false, false
+            +- CTERelationRef xxxx, true, [m#xL], false, false, false
 
 
 -- !query
@@ -2008,10 +2110,10 @@
       +- Union false, false
          :- Project [n#x]
          :  +- SubqueryAlias t1
-         :     +- CTERelationRef xxxx, true, [n#x], false, false
+         :     +- CTERelationRef xxxx, true, [n#x], false, false, false
          +- Project [n#x]
             +- SubqueryAlias t1
-               +- CTERelationRef xxxx, true, [n#x], false, false
+               +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -2130,4 +2232,4 @@
 :                    +- UnionLoopRef xxxx, [n#x], false
 +- Project [n#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out
index 8d8d978..3b2cafd 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out
@@ -86,7 +86,7 @@
 :                       +- LocalRelation [id#x]
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x], false, false
+      +- CTERelationRef xxxx, true, [1#x], false, false, false
 
 
 -- !query
@@ -147,13 +147,13 @@
 :  +- SubqueryAlias t2
 :     +- Project [2 AS 2#x]
 :        +- SubqueryAlias t1
-:           +- CTERelationRef xxxx, true, [id#x], false, false
+:           +- CTERelationRef xxxx, true, [id#x], false, false, false
 +- Project [id#x, 2#x]
    +- Join Cross
       :- SubqueryAlias t1
-      :  +- CTERelationRef xxxx, true, [id#x], false, false
+      :  +- CTERelationRef xxxx, true, [id#x], false, false, false
       +- SubqueryAlias t2
-         +- CTERelationRef xxxx, true, [2#x], false, false
+         +- CTERelationRef xxxx, true, [2#x], false, false, false
 
 
 -- !query
@@ -191,10 +191,10 @@
    +- Join Cross
       :- SubqueryAlias t1
       :  +- SubqueryAlias CTE1
-      :     +- CTERelationRef xxxx, true, [id#x], false, false
+      :     +- CTERelationRef xxxx, true, [id#x], false, false, false
       +- SubqueryAlias t2
          +- SubqueryAlias CTE1
-            +- CTERelationRef xxxx, true, [id#x], false, false
+            +- CTERelationRef xxxx, true, [id#x], false, false, false
 
 
 -- !query
@@ -210,7 +210,7 @@
 +- Project [x#x]
    +- Filter (x#x = 1)
       +- SubqueryAlias t
-         +- CTERelationRef xxxx, true, [x#x], false, false, 1
+         +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 
 
 -- !query
@@ -226,7 +226,7 @@
 +- Project [x#x, y#x]
    +- Filter ((x#x = 1) AND (y#x = 2))
       +- SubqueryAlias t
-         +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1
+         +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1, false
 
 
 -- !query
@@ -241,7 +241,7 @@
 :           +- OneRowRelation
 +- Project [x#x, x#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [x#x, x#x], false, false, 1
+      +- CTERelationRef xxxx, true, [x#x, x#x], false, false, 1, false
 
 
 -- !query
@@ -344,46 +344,46 @@
 :     +- Project [c8#x AS c7#x]
 :        +- Project [c8#x]
 :           +- SubqueryAlias w8
-:              +- CTERelationRef xxxx, true, [c8#x], false, false, 1
+:              +- CTERelationRef xxxx, true, [c8#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w6
 :     +- Project [c7#x AS c6#x]
 :        +- Project [c7#x]
 :           +- SubqueryAlias w7
-:              +- CTERelationRef xxxx, true, [c7#x], false, false, 1
+:              +- CTERelationRef xxxx, true, [c7#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w5
 :     +- Project [c6#x AS c5#x]
 :        +- Project [c6#x]
 :           +- SubqueryAlias w6
-:              +- CTERelationRef xxxx, true, [c6#x], false, false, 1
+:              +- CTERelationRef xxxx, true, [c6#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w4
 :     +- Project [c5#x AS c4#x]
 :        +- Project [c5#x]
 :           +- SubqueryAlias w5
-:              +- CTERelationRef xxxx, true, [c5#x], false, false, 1
+:              +- CTERelationRef xxxx, true, [c5#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w3
 :     +- Project [c4#x AS c3#x]
 :        +- Project [c4#x]
 :           +- SubqueryAlias w4
-:              +- CTERelationRef xxxx, true, [c4#x], false, false, 1
+:              +- CTERelationRef xxxx, true, [c4#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w2
 :     +- Project [c3#x AS c2#x]
 :        +- Project [c3#x]
 :           +- SubqueryAlias w3
-:              +- CTERelationRef xxxx, true, [c3#x], false, false, 1
+:              +- CTERelationRef xxxx, true, [c3#x], false, false, 1, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w1
 :     +- Project [c2#x AS c1#x]
 :        +- Project [c2#x]
 :           +- SubqueryAlias w2
-:              +- CTERelationRef xxxx, true, [c2#x], false, false, 1
+:              +- CTERelationRef xxxx, true, [c2#x], false, false, 1, false
 +- Project [c1#x]
    +- SubqueryAlias w1
-      +- CTERelationRef xxxx, true, [c1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false
 
 
 -- !query
@@ -420,7 +420,7 @@
 +- Project [42#x, 10#x]
    +- Join Inner
       :- SubqueryAlias same_name
-      :  +- CTERelationRef xxxx, true, [42#x], false, false, 1
+      :  +- CTERelationRef xxxx, true, [42#x], false, false, 1, false
       +- SubqueryAlias same_name
          +- Project [10 AS 10#x]
             +- OneRowRelation
@@ -459,7 +459,7 @@
 :        +- OneRowRelation
 +- Project [x#x, typeof(x#x) AS typeof(x)#x]
    +- SubqueryAlias q
-      +- CTERelationRef xxxx, true, [x#x], false, false, 1
+      +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 
 
 -- !query
@@ -519,7 +519,7 @@
       :        +- OneRowRelation
       +- Project [(x#x + 1) AS y#x]
          +- SubqueryAlias q
-            +- CTERelationRef xxxx, true, [x#x], false, false, 1
+            +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 
 
 -- !query
@@ -533,7 +533,7 @@
 :     :        +- OneRowRelation
 :     +- Project [x#x]
 :        +- SubqueryAlias q
-:           +- CTERelationRef xxxx, true, [x#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 +- OneRowRelation
 
 
@@ -548,7 +548,7 @@
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias q
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- OneRowRelation
 
 
@@ -596,14 +596,14 @@
       :- Join Inner
       :  :- SubqueryAlias x
       :  :  +- SubqueryAlias T1
-      :  :     +- CTERelationRef xxxx, true, [a#x], false, false, 1
+      :  :     +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
       :  +- SubqueryAlias y
       :     +- Project [b#x]
       :        +- SubqueryAlias T1
-      :           +- CTERelationRef xxxx, true, [b#x], false, false, 1
+      :           +- CTERelationRef xxxx, true, [b#x], false, false, 1, false
       +- SubqueryAlias z
          +- SubqueryAlias T1
-            +- CTERelationRef xxxx, true, [a#x], false, false, 1
+            +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
 
 
 -- !query
@@ -631,9 +631,9 @@
       +- Project [c#x, a#x]
          +- Join Inner
             :- SubqueryAlias ttTT
-            :  +- CTERelationRef xxxx, true, [c#x], false, false, 1
+            :  +- CTERelationRef xxxx, true, [c#x], false, false, 1, false
             +- SubqueryAlias tttT_2
-               +- CTERelationRef xxxx, true, [a#x], false, false, 1
+               +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
 
 
 -- !query
@@ -649,7 +649,7 @@
 :     :        +- OneRowRelation
 :     +- Project [x#x]
 :        +- SubqueryAlias q
-:           +- CTERelationRef xxxx, true, [x#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 +- SubqueryAlias T
    +- Project [1 AS x#x, 2 AS y#x]
       +- OneRowRelation
@@ -668,7 +668,7 @@
 :     :        +- OneRowRelation
 :     +- Project [((outer(x#x) + outer(y#x)) + z#x) AS ((outer(T.x) + outer(T.y)) + z)#x]
 :        +- SubqueryAlias q
-:           +- CTERelationRef xxxx, true, [z#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [z#x], false, false, 1, false
 +- SubqueryAlias T
    +- Project [1 AS x#x, 2 AS y#x]
       +- OneRowRelation
@@ -688,12 +688,12 @@
 :  +- SubqueryAlias q2
 :     +- Project [x#x]
 :        +- SubqueryAlias q1
-:           +- CTERelationRef xxxx, true, [x#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 +- Project [x#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [x#x]
          +- SubqueryAlias q2
-            +- CTERelationRef xxxx, true, [x#x], false, false, 1
+            +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 
 
 -- !query
@@ -710,12 +710,12 @@
 :  +- SubqueryAlias q1
 :     +- Project [(x#x + 1) AS (x + 1)#x]
 :        +- SubqueryAlias q1
-:           +- CTERelationRef xxxx, true, [x#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 +- Project [(x + 1)#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [(x + 1)#x]
          +- SubqueryAlias q1
-            +- CTERelationRef xxxx, true, [(x + 1)#x], false, false, 1
+            +- CTERelationRef xxxx, true, [(x + 1)#x], false, false, 1, false
 
 
 -- !query
@@ -757,9 +757,9 @@
       :  +- Aggregate [max(j#x) AS max(j)#x]
       :     +- SubqueryAlias cte2
       :        +- SubqueryAlias cte1
-      :           +- CTERelationRef xxxx, true, [j#x], false, false
+      :           +- CTERelationRef xxxx, true, [j#x], false, false, false
       +- SubqueryAlias cte1
-         +- CTERelationRef xxxx, true, [j#x], false, false
+         +- CTERelationRef xxxx, true, [j#x], false, false, false
 
 
 -- !query
@@ -827,10 +827,10 @@
       +- Union false, false
          :- Project [1#x]
          :  +- SubqueryAlias cte
-         :     +- CTERelationRef xxxx, true, [1#x], false, false, 1
+         :     +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
          +- Project [2#x]
             +- SubqueryAlias cte
-               +- CTERelationRef xxxx, true, [2#x], false, false, 1
+               +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
 
 
 -- !query
@@ -892,7 +892,7 @@
 :     :     :        +- OneRowRelation
 :     :     +- Project [1#x]
 :     :        +- SubqueryAlias cte
-:     :           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:     :           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :     +- LocalRelation [col1#x]
 +- Project [col1#x]
    +- Filter exists#x []
@@ -903,7 +903,7 @@
       :     :        +- OneRowRelation
       :     +- Project [1#x]
       :        +- SubqueryAlias cte
-      :           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      :           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
       +- LocalRelation [col1#x]
 
 
@@ -943,7 +943,7 @@
 :        :     :     :        +- OneRowRelation
 :        :     :     +- Project [1#x]
 :        :     :        +- SubqueryAlias cte
-:        :     :           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:        :     :           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :        :     +- LocalRelation [col1#x]
 :        +- Project [col1#x]
 :           +- Filter exists#x []
@@ -954,11 +954,11 @@
 :              :     :        +- OneRowRelation
 :              :     +- Project [1#x]
 :              :        +- SubqueryAlias cte
-:              :           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:              :           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :              +- LocalRelation [col1#x]
 +- Project [col1#x]
    +- SubqueryAlias cte
-      +- CTERelationRef xxxx, true, [col1#x], false, false, 2
+      +- CTERelationRef xxxx, true, [col1#x], false, false, 2, false
 
 
 -- !query
@@ -1000,7 +1000,7 @@
          :     :     :        +- OneRowRelation
          :     :     +- Project [1#x]
          :     :        +- SubqueryAlias cte
-         :     :           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+         :     :           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
          :     +- LocalRelation [col1#x]
          +- Project [col1#x]
             +- Filter exists#x []
@@ -1011,7 +1011,7 @@
                :     :        +- OneRowRelation
                :     +- Project [1#x]
                :        +- SubqueryAlias cte
-               :           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+               :           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
                +- LocalRelation [col1#x]
 
 
@@ -1028,7 +1028,7 @@
 :        +- OneRowRelation
 +- Project [1#x]
    +- SubqueryAlias `a.b.c`
-      +- CTERelationRef xxxx, true, [1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out
index 51569e6..12f48c1 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out
@@ -438,7 +438,7 @@
       :           +- OneRowRelation
       +- Project [a#x]
          +- SubqueryAlias v
-            +- CTERelationRef xxxx, true, [a#x], false, false, 1
+            +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
 
 
 -- !query
@@ -458,7 +458,7 @@
                   :           +- OneRowRelation
                   +- Project [a#x]
                      +- SubqueryAlias v
-                        +- CTERelationRef xxxx, true, [a#x], false, false, 1
+                        +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out
index a3fe0be..55d7710 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-alias.sql.out
@@ -368,7 +368,7 @@
 :  :  :        +- OneRowRelation
 :  :  +- Project [1#x]
 :  :     +- SubqueryAlias cte
-:  :        +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:  :        +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :  +- WithCTE
 :     :- CTERelationDef xxxx, false
 :     :  +- SubqueryAlias cte
@@ -376,7 +376,7 @@
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias cte
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- LocalRelation [col1#x]
 
 
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out
index 7837ba4..b6b8fdf 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out
@@ -416,7 +416,7 @@
 :  :  :        +- OneRowRelation
 :  :  +- Project [1#x]
 :  :     +- SubqueryAlias cte
-:  :        +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:  :        +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :  +- WithCTE
 :     :- CTERelationDef xxxx, false
 :     :  +- SubqueryAlias cte
@@ -424,7 +424,7 @@
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias cte
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- LocalRelation [col1#x]
 
 
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out
index 430e645..72d8f64 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out
@@ -489,7 +489,7 @@
 :  :  :        +- OneRowRelation
 :  :  +- Project [1#x]
 :  :     +- SubqueryAlias cte
-:  :        +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:  :        +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 :  +- WithCTE
 :     :- CTERelationDef xxxx, false
 :     :  +- SubqueryAlias cte
@@ -497,7 +497,7 @@
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias cte
-:           +- CTERelationRef xxxx, true, [1#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [1#x], false, false, 1, false
 +- LocalRelation [col1#x]
 
 
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
index 38b28f2..8f0676f 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
@@ -1022,7 +1022,7 @@
 :        +- LocalRelation [col1#x, col2#x]
 +- Aggregate [max(c1#x) AS max(c1)#x]
    +- SubqueryAlias T
-      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2
+      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2, false
 
 
 -- !query
@@ -1041,7 +1041,7 @@
 :        +- LocalRelation [col1#x, col2#x]
 +- Aggregate [max(c1#x) AS max(c1)#x]
    +- SubqueryAlias T
-      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2
+      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2, false
 
 
 -- !query
@@ -1055,7 +1055,7 @@
 :        +- LocalRelation [col1#x, col2#x]
 +- Aggregate [max(c1#x) AS max(c1)#x]
    +- SubqueryAlias ABC
-      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2
+      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out
index 390fcf5..e5acfcb 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out
@@ -1377,10 +1377,10 @@
 :           :                 +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
 :           :                    +- LocalRelation [col1#x, col2#x]
 :           +- SubqueryAlias cte1
-:              +- CTERelationRef xxxx, true, [c1#x], false, false
+:              +- CTERelationRef xxxx, true, [c1#x], false, false, false
 +- Project [c1#x, c2#x]
    +- SubqueryAlias cte2
-      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+      +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out
index 7a7d7aa..6fe453a5 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out
@@ -20,7 +20,7 @@
 :              +- LocalRelation [col1#x]
 +- Project [len(c1#x) AS len(c1)#x, regexp_count(c1#x, a) AS regexp_count(c1, a)#x, regexp_count(c1#x, b) AS regexp_count(c1, b)#x, regexp_count(c1#x, A) AS regexp_count(c1, A)#x, regexp_count(c1#x, B) AS regexp_count(c1, B)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [c1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false
 
 
 -- !query
@@ -37,7 +37,7 @@
 , $) AS replace(replace(c1,  , ), 
 , $)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [c1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false
 
 
 -- !query
@@ -61,7 +61,7 @@
 :              +- LocalRelation [col1#x]
 +- Project [len(c1#x) AS len(c1)#x, regexp_count(c1#x, a) AS regexp_count(c1, a)#x, regexp_count(c1#x, b) AS regexp_count(c1, b)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [c1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false
 
 
 -- !query
@@ -85,7 +85,7 @@
 :              +- LocalRelation [col1#x]
 +- Project [len(c1#x) AS len(c1)#x, regexp_count(c1#x, a) AS regexp_count(c1, a)#x, regexp_count(c1#x, xbc) AS regexp_count(c1, xbc)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [c1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false
 
 
 -- !query
@@ -102,7 +102,7 @@
 , $) AS replace(replace(c1,  , ), 
 , $)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [c1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out
index 1d78d88..e22fc1f 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out
@@ -35,7 +35,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false
+      +- CTERelationRef xxxx, true, [col#x], false, false, false
 
 
 -- !query
@@ -54,7 +54,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false
+      +- CTERelationRef xxxx, true, [col#x], false, false, false
 
 
 -- !query
@@ -73,7 +73,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false
+      +- CTERelationRef xxxx, true, [col#x], false, false, false
 
 
 -- !query
@@ -105,7 +105,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x, regexp_count(col#x, c) AS regexp_count(col, c)#x, regexp_count(col#x, d) AS regexp_count(col, d)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false
+      +- CTERelationRef xxxx, true, [col#x], false, false, false
 
 
 -- !query
@@ -124,7 +124,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -143,7 +143,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(col#x, a) AS regexp_count(col, a)#x, regexp_count(col#x, b) AS regexp_count(col, b)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -213,7 +213,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [concat(cast((col#x = baba) as string), cast((col#x = bbaa) as string)) AS concat((col = baba), (col = bbaa))#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -232,7 +232,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [concat(cast((col#x = b|a|b|a) as string), cast((col#x = b|b|a|a) as string)) AS concat((col = b|a|b|a), (col = b|b|a|a))#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -283,7 +283,7 @@
 :              +- LocalRelation [col1#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(cast(col#x as string), cast(0xDEAD as string)) AS regexp_count(col, X'DEAD')#x, regexp_count(cast(col#x as string), cast(0xBEEF as string)) AS regexp_count(col, X'BEEF')#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -298,7 +298,7 @@
 :              +- LocalRelation [col1#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(cast(col#x as string), cast(0xDEAD as string)) AS regexp_count(col, X'DEAD')#x, regexp_count(cast(col#x as string), cast(0xBEEF as string)) AS regexp_count(col, X'BEEF')#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -313,7 +313,7 @@
 :              +- LocalRelation [col1#x]
 +- Project [len(col#x) AS len(col)#x, regexp_count(cast(col#x as string), cast(0x42 as string)) AS regexp_count(col, X'42')#x, regexp_count(cast(col#x as string), cast(0xDEAD as string)) AS regexp_count(col, X'DEAD')#x, regexp_count(cast(col#x as string), cast(0xBEEF as string)) AS regexp_count(col, X'BEEF')#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col#x], false, false, 1, false
 
 
 -- !query
@@ -332,7 +332,7 @@
 :                          +- LocalRelation [col1#x, col2#x]
 +- Project [len(col1#x) AS len(col1)#x, regexp_count(col1#x, 1) AS regexp_count(col1, 1)#x, regexp_count(col1#x, 2) AS regexp_count(col1, 2)#x, regexp_count(col1#x, 3) AS regexp_count(col1, 3)#x, len(col2#x) AS len(col2)#x, regexp_count(col2#x, true) AS regexp_count(col2, true)#x, regexp_count(col1#x, false) AS regexp_count(col1, false)#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [col1#x, col2#x], false, false, 1
+      +- CTERelationRef xxxx, true, [col1#x, col2#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out
index 6bfbe53..6ae281b 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out
@@ -47,7 +47,7 @@
    +- Filter (id#xL > scalar-subquery#x [])
       :  +- Aggregate [max(id#xL) AS max(id)#xL]
       :     +- SubqueryAlias tmp
-      :        +- CTERelationRef xxxx, true, [id#xL], false, false, 2
+      :        +- CTERelationRef xxxx, true, [id#xL], false, false, 2, false
       +- Range (0, 3, step=1)
 
 
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
index bc6dc82..4df9bef 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
@@ -99,7 +99,7 @@
       +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST]
          +- Project [x#xL]
             +- SubqueryAlias cte
-               +- CTERelationRef xxxx, true, [x#xL], false, false, 18
+               +- CTERelationRef xxxx, true, [x#xL], false, false, 18, false
 
 
 -- !query
@@ -121,7 +121,7 @@
       +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST]
          +- Project [x#xL]
             +- SubqueryAlias cte
-               +- CTERelationRef xxxx, true, [x#xL], false, false, 18
+               +- CTERelationRef xxxx, true, [x#xL], false, false, 18, false
 
 
 -- !query
@@ -154,7 +154,7 @@
       +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST]
          +- Project [x#xL]
             +- SubqueryAlias cte
-               +- CTERelationRef xxxx, true, [x#xL], false, false, 26
+               +- CTERelationRef xxxx, true, [x#xL], false, false, 26, false
 
 
 -- !query
@@ -187,7 +187,7 @@
       +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST]
          +- Project [x#xL]
             +- SubqueryAlias cte
-               +- CTERelationRef xxxx, true, [x#xL], false, false, 26
+               +- CTERelationRef xxxx, true, [x#xL], false, false, 26, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
index edbdcba..e56f8ac 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
@@ -12,10 +12,10 @@
 +- Project [x#x, y#x, x#x, y#x]
    +- Join Inner
       :- SubqueryAlias q1
-      :  +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1
+      :  +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1, false
       +- SubqueryAlias q2
          +- SubqueryAlias q1
-            +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1
+            +- CTERelationRef xxxx, true, [x#x, y#x], false, false, 1, false
 
 
 -- !query
@@ -50,7 +50,7 @@
 :                       +- UnionLoopRef xxxx, [col1#x], false
 +- Aggregate [sum(n#x) AS sum(n)#xL]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -76,7 +76,7 @@
 :                       +- UnionLoopRef xxxx, [scalarsubquery()#x], false
 +- Project [n#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -107,7 +107,7 @@
       :                       +- UnionLoopRef xxxx, [col1#x], false
       +- Project [n#x]
          +- SubqueryAlias nums
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -130,7 +130,7 @@
             :                       +- UnionLoopRef xxxx, [col1#x], false
             +- Project [n#x]
                +- SubqueryAlias nums
-                  +- CTERelationRef xxxx, true, [n#x], false, false
+                  +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -161,7 +161,7 @@
       :                       +- UnionLoopRef xxxx, [col1#x], false
       +- Project [n#x]
          +- SubqueryAlias nums
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -184,7 +184,7 @@
             :                       +- UnionLoopRef xxxx, [col1#x], false
             +- Project [n#x]
                +- SubqueryAlias nums
-                  +- CTERelationRef xxxx, true, [n#x], false, false
+                  +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -229,7 +229,7 @@
    +- LocalLimit 10
       +- Project [n#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -264,7 +264,7 @@
 :        +- OneRowRelation
 +- Project [x#x]
    +- SubqueryAlias q
-      +- CTERelationRef xxxx, true, [x#x], false, false, 1
+      +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 
 
 -- !query
@@ -289,7 +289,7 @@
 :                       +- UnionLoopRef xxxx, [foo#x], false
 +- Project [n#x AS is_text#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -417,7 +417,7 @@
 +- Sort [name#x ASC NULLS FIRST], true
    +- Project [root_name#x, id#x, parent_department#x, name#x]
       +- SubqueryAlias subdepartment
-         +- CTERelationRef xxxx, true, [root_name#x, id#x, parent_department#x, name#x], false, false
+         +- CTERelationRef xxxx, true, [root_name#x, id#x, parent_department#x, name#x], false, false, false
 
 
 -- !query
@@ -454,7 +454,7 @@
 +- Sort [name#x ASC NULLS FIRST], true
    +- Project [level#x, id#x, parent_department#x, name#x]
       +- SubqueryAlias subdepartment
-         +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false
+         +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false, false
 
 
 -- !query
@@ -492,7 +492,7 @@
    +- Project [level#x, id#x, parent_department#x, name#x]
       +- Filter (level#x >= 2)
          +- SubqueryAlias subdepartment
-            +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false
+            +- CTERelationRef xxxx, true, [level#x, id#x, parent_department#x, name#x], false, false, false
 
 
 -- !query
@@ -512,7 +512,7 @@
 +- Sort [name#x ASC NULLS FIRST], true
    +- Project [id#x, parent_department#x, name#x]
       +- SubqueryAlias subdepartment
-         +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false
+         +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false, false
 
 
 -- !query
@@ -554,7 +554,7 @@
    :              +- Project [n#x]
    :                 +- Filter (n#x < 50000)
    :                    +- SubqueryAlias t
-   :                       +- CTERelationRef xxxx, true, [n#x], false, false
+   :                       +- CTERelationRef xxxx, true, [n#x], false, false, false
    +- SubqueryAlias t
       +- WithCTE
          :- CTERelationDef xxxx, false
@@ -570,7 +570,7 @@
          :                       +- UnionLoopRef xxxx, [1#x], false
          +- Project [n#x]
             +- SubqueryAlias t
-               +- CTERelationRef xxxx, true, [n#x], false, false
+               +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -597,9 +597,9 @@
       :  +- Aggregate [(cast(sum(y#xL) as double) / cast(100 as double)) AS (sum(y) / 100)#x]
       :     +- SubqueryAlias qsub
       :        +- SubqueryAlias q1
-      :           +- CTERelationRef xxxx, true, [x#x, y#xL], false, false
+      :           +- CTERelationRef xxxx, true, [x#x, y#xL], false, false, false
       +- SubqueryAlias q1
-         +- CTERelationRef xxxx, true, [x#x, y#xL], false, false
+         +- CTERelationRef xxxx, true, [x#x, y#xL], false, false, false
 
 
 -- !query
@@ -640,7 +640,7 @@
       :                       +- UnionLoopRef xxxx, [id#x, parent_department#x, name#x], false
       +- Project [id#x, parent_department#x, name#x]
          +- SubqueryAlias subdepartment
-            +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false
+            +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false, false
 
 
 -- !query
@@ -670,7 +670,7 @@
                :                       +- UnionLoopRef xxxx, [id#x, parent_department#x, name#x], false
                +- Project [id#x, parent_department#x, name#x]
                   +- SubqueryAlias subdepartment
-                     +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false
+                     +- CTERelationRef xxxx, true, [id#x, parent_department#x, name#x], false, false, false
 
 
 -- !query
@@ -701,7 +701,7 @@
       :                       +- UnionLoopRef xxxx, [col1#x], false
       +- Aggregate [sum(n#x) AS sum#xL]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [n#x], false, false
+            +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -724,7 +724,7 @@
             :                       +- UnionLoopRef xxxx, [col1#x], false
             +- Aggregate [sum(n#x) AS sum#xL]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [n#x], false, false
+                  +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -756,7 +756,7 @@
 :                       +- UnionLoopRef xxxx, [col1#x, col2#x], false
 +- Project [i#x, j#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [i#x, j#x], false, false
+      +- CTERelationRef xxxx, true, [i#x, j#x], false, false, false
 
 
 -- !query
@@ -809,10 +809,10 @@
       +- Join Inner, (((path#x[0] = path#x[0]) AND (size(path#x, false) = 1)) AND (size(path#x, false) > 1))
          :- SubqueryAlias t1
          :  +- SubqueryAlias t
-         :     +- CTERelationRef xxxx, true, [id#x, path#x], false, false
+         :     +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false
          +- SubqueryAlias t2
             +- SubqueryAlias t
-               +- CTERelationRef xxxx, true, [id#x, path#x], false, false
+               +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false
 
 
 -- !query
@@ -847,10 +847,10 @@
       +- Join Inner, (((path#x[0] = path#x[0]) AND (size(path#x, false) = 1)) AND (size(path#x, false) > 1))
          :- SubqueryAlias t1
          :  +- SubqueryAlias t
-         :     +- CTERelationRef xxxx, true, [id#x, path#x], false, false
+         :     +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false
          +- SubqueryAlias t2
             +- SubqueryAlias t
-               +- CTERelationRef xxxx, true, [id#x, path#x], false, false
+               +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false
 
 
 -- !query
@@ -880,10 +880,10 @@
    +- Join Inner, (id#x = id#x)
       :- SubqueryAlias t1
       :  +- SubqueryAlias t
-      :     +- CTERelationRef xxxx, true, [id#x, path#x], false, false
+      :     +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false
       +- SubqueryAlias t2
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [id#x, path#x], false, false
+            +- CTERelationRef xxxx, true, [id#x, path#x], false, false, false
 
 
 -- !query
@@ -937,7 +937,7 @@
 :                             +- UnionLoopRef xxxx, [f#x, t#x, label#x, array(struct(f, t))#x, false#x], false
 +- Project [f#x, t#x, label#x, path#x, cycle#x]
    +- SubqueryAlias search_graph
-      +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false
+      +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false, false
 
 
 -- !query
@@ -972,7 +972,7 @@
 +- Sort [path#x ASC NULLS FIRST], true
    +- Project [f#x, t#x, label#x, path#x, cycle#x]
       +- SubqueryAlias search_graph
-         +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false
+         +- CTERelationRef xxxx, true, [f#x, t#x, label#x, path#x, cycle#x], false, false, false
 
 
 -- !query
@@ -992,7 +992,7 @@
 :        +- UnionLoop xxxx
 :           :- Project [id#x]
 :           :  +- SubqueryAlias y
-:           :     +- CTERelationRef xxxx, true, [id#x], false, false, 1
+:           :     +- CTERelationRef xxxx, true, [id#x], false, false, 1, false
 :           +- Project [(id#x + 1) AS (id + 1)#x]
 :              +- Filter (id#x < 5)
 :                 +- SubqueryAlias x
@@ -1000,7 +1000,7 @@
 :                       +- UnionLoopRef xxxx, [id#x], false
 +- Project [id#x]
    +- SubqueryAlias x
-      +- CTERelationRef xxxx, true, [id#x], false, false
+      +- CTERelationRef xxxx, true, [id#x], false, false, false
 
 
 -- !query
@@ -1031,17 +1031,17 @@
 :        +- Union false, false
 :           :- Project [id#x]
 :           :  +- SubqueryAlias x
-:           :     +- CTERelationRef xxxx, true, [id#x], false, false
+:           :     +- CTERelationRef xxxx, true, [id#x], false, false, false
 :           +- Project [id#x]
 :              +- SubqueryAlias x
-:                 +- CTERelationRef xxxx, true, [id#x], false, false
+:                 +- CTERelationRef xxxx, true, [id#x], false, false, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias z
 :     +- Project [id#x AS id#x]
 :        +- UnionLoop xxxx
 :           :- Project [id#x]
 :           :  +- SubqueryAlias x
-:           :     +- CTERelationRef xxxx, true, [id#x], false, false
+:           :     +- CTERelationRef xxxx, true, [id#x], false, false, false
 :           +- Project [(id#x + 1) AS (id + 1)#x]
 :              +- Filter (id#x < 10)
 :                 +- SubqueryAlias z
@@ -1049,7 +1049,7 @@
 :                       +- UnionLoopRef xxxx, [id#x], false
 +- Project [id#x]
    +- SubqueryAlias z
-      +- CTERelationRef xxxx, true, [id#x], false, false
+      +- CTERelationRef xxxx, true, [id#x], false, false, false
 
 
 -- !query
@@ -1080,17 +1080,17 @@
 :        +- Union false, false
 :           :- Project [id#x]
 :           :  +- SubqueryAlias x
-:           :     +- CTERelationRef xxxx, true, [id#x], false, false
+:           :     +- CTERelationRef xxxx, true, [id#x], false, false, false
 :           +- Project [id#x]
 :              +- SubqueryAlias x
-:                 +- CTERelationRef xxxx, true, [id#x], false, false
+:                 +- CTERelationRef xxxx, true, [id#x], false, false, false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias z
 :     +- Project [id#x AS id#x]
 :        +- UnionLoop xxxx
 :           :- Project [id#x]
 :           :  +- SubqueryAlias y
-:           :     +- CTERelationRef xxxx, true, [id#x], false, false
+:           :     +- CTERelationRef xxxx, true, [id#x], false, false, false
 :           +- Project [(id#x + 1) AS (id + 1)#x]
 :              +- Filter (id#x < 10)
 :                 +- SubqueryAlias z
@@ -1098,7 +1098,7 @@
 :                       +- UnionLoopRef xxxx, [id#x], false
 +- Project [id#x]
    +- SubqueryAlias z
-      +- CTERelationRef xxxx, true, [id#x], false, false
+      +- CTERelationRef xxxx, true, [id#x], false, false, false
 
 
 -- !query
@@ -1268,7 +1268,7 @@
 :                       +- UnionLoopRef xxxx, [1#x], false
 +- Project [n#x]
    +- SubqueryAlias x
-      +- CTERelationRef xxxx, true, [n#x], false, false
+      +- CTERelationRef xxxx, true, [n#x], false, false, false
 
 
 -- !query
@@ -1338,7 +1338,7 @@
 :                       +- UnionLoopRef xxxx, [col1#x], false
 +- Project [id#x]
    +- SubqueryAlias x
-      +- CTERelationRef xxxx, true, [id#x], false, false
+      +- CTERelationRef xxxx, true, [id#x], false, false, false
 
 
 -- !query
@@ -1392,7 +1392,7 @@
 :                          +- UnionLoopRef xxxx, [col1#x], false
 +- Project [i#x]
    +- SubqueryAlias foo
-      +- CTERelationRef xxxx, true, [i#x], false, false
+      +- CTERelationRef xxxx, true, [i#x], false, false, false
 
 
 -- !query
@@ -1426,7 +1426,7 @@
 :                                +- UnionLoopRef xxxx, [col1#x], false
 +- Project [i#x]
    +- SubqueryAlias foo
-      +- CTERelationRef xxxx, true, [i#x], false, false
+      +- CTERelationRef xxxx, true, [i#x], false, false, false
 
 
 -- !query
@@ -1457,7 +1457,7 @@
 :                          +- UnionLoopRef xxxx, [col1#x], false
 +- Project [i#x]
    +- SubqueryAlias foo
-      +- CTERelationRef xxxx, true, [i#x], false, false
+      +- CTERelationRef xxxx, true, [i#x], false, false, false
 
 
 -- !query
@@ -1488,7 +1488,7 @@
 :                          +- UnionLoopRef xxxx, [col1#x], false
 +- Project [i#x]
    +- SubqueryAlias foo
-      +- CTERelationRef xxxx, true, [i#x], false, false
+      +- CTERelationRef xxxx, true, [i#x], false, false, false
 
 
 -- !query
@@ -1522,7 +1522,7 @@
    +- SubqueryAlias q
       +- Project [foo#x]
          +- SubqueryAlias cte
-            +- CTERelationRef xxxx, true, [foo#x], false, false, 1
+            +- CTERelationRef xxxx, true, [foo#x], false, false, 1, false
 
 
 -- !query
@@ -1556,7 +1556,7 @@
 :           +- UnionLoop xxxx
 :              :- Project [i#x]
 :              :  +- SubqueryAlias s
-:              :     +- CTERelationRef xxxx, true, [i#x], false, false
+:              :     +- CTERelationRef xxxx, true, [i#x], false, false, false
 :              +- Project [(j#x + 1) AS (j + 1)#x]
 :                 +- Filter (j#x < 10)
 :                    +- SubqueryAlias t
@@ -1564,7 +1564,7 @@
 :                          +- UnionLoopRef xxxx, [i#x], false
 +- Project [j#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [j#x], false, false
+      +- CTERelationRef xxxx, true, [j#x], false, false, false
 
 
 -- !query
@@ -1592,13 +1592,13 @@
 :                 +- Union false, false
 :                    :- Project [2#x]
 :                    :  +- SubqueryAlias innermost
-:                    :     +- CTERelationRef xxxx, true, [2#x], false, false, 1
+:                    :     +- CTERelationRef xxxx, true, [2#x], false, false, 1, false
 :                    +- Project [3 AS 3#x]
 :                       +- OneRowRelation
 +- Sort [x#x ASC NULLS FIRST], true
    +- Project [x#x]
       +- SubqueryAlias outermost
-         +- CTERelationRef xxxx, true, [x#x], false, false, 3
+         +- CTERelationRef xxxx, true, [x#x], false, false, 3, false
 
 
 -- !query
@@ -1826,7 +1826,7 @@
 :        +- OneRowRelation
 +- Project [x#x]
    +- SubqueryAlias ordinality
-      +- CTERelationRef xxxx, true, [x#x], false, false, 1
+      +- CTERelationRef xxxx, true, [x#x], false, false, 1, false
 
 
 -- !query
@@ -1867,7 +1867,7 @@
       :        +- OneRowRelation
       +- Project [42#x]
          +- SubqueryAlias test
-            +- CTERelationRef xxxx, true, [42#x], false, false, 1
+            +- CTERelationRef xxxx, true, [42#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out
index 7d23f27..8e8ed57 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out
@@ -2221,7 +2221,7 @@
 :        +- OneRowRelation
 +- Project [c1#x AS 1#x]
    +- SubqueryAlias v1
-      +- CTERelationRef xxxx, true, [c1#x], false, false, 1
+      +- CTERelationRef xxxx, true, [c1#x], false, false, 1, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out
index 2b8a47c..4fd3b58 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out
@@ -2803,7 +2803,7 @@
 :                    +- LocalRelation [col1#x, col2#x]
 +- Project [spark_catalog.default.foo3_1a(c1, c2)#x]
    +- SubqueryAlias cte
-      +- CTERelationRef xxxx, true, [spark_catalog.default.foo3_1a(c1, c2)#x], false, false
+      +- CTERelationRef xxxx, true, [spark_catalog.default.foo3_1a(c1, c2)#x], false, false, false
 
 
 -- !query
@@ -3528,7 +3528,7 @@
 :     :        +- OneRowRelation
 :     +- Project [CASE WHEN (isnull(outer(occurrences#x)) OR (size(outer(occurrences#x), false) = 0)) THEN cast(null as string) ELSE sort_array(diffs#x, true)[0].id END AS id#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [diffs#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [diffs#x], false, false, 1, false
 +- Project [cast(array(struct(col1, 2022-01-01 10:11:12, col2, 1), struct(col1, 2022-01-01 10:11:15, col2, 2)) as array<struct<start_time:timestamp,occurrence_id:string>>) AS occurrences#x, cast(2022-01-01 as timestamp) AS instance_start_time#x]
    +- OneRowRelation
 
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out
index abaf6a2..aaa098d 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out
@@ -133,7 +133,7 @@
       :     +- Filter (outer(emp_name#x) = emp_name#x)
       :        +- SubqueryAlias b
       :           +- SubqueryAlias bonus_cte
-      :              +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false, false
+      :              +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false, false, false
       +- SubqueryAlias a
          +- SubqueryAlias bonus
             +- View (`BONUS`, [emp_name#x, bonus_amt#x])
@@ -189,10 +189,10 @@
       :        +- Join Inner, (dept_id#x = dept_id#x)
       :           :- SubqueryAlias a
       :           :  +- SubqueryAlias emp_cte
-      :           :     +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false
+      :           :     +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false, false
       :           +- SubqueryAlias b
       :              +- SubqueryAlias dept_cte
-      :                 +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false
+      :                 +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false, false
       +- SubqueryAlias bonus
          +- View (`BONUS`, [emp_name#x, bonus_amt#x])
             +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x]
@@ -253,10 +253,10 @@
          :        +- Join LeftOuter, (dept_id#x = dept_id#x)
          :           :- SubqueryAlias a
          :           :  +- SubqueryAlias emp_cte
-         :           :     +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false
+         :           :     +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false, false
          :           +- SubqueryAlias b
          :              +- SubqueryAlias dept_cte
-         :                 +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false
+         :                 +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false, false
          +- Join Inner
             :- Join Inner
             :  :- SubqueryAlias b
@@ -268,7 +268,7 @@
             :  :                 +- LocalRelation [emp_name#x, bonus_amt#x]
             :  +- SubqueryAlias e
             :     +- SubqueryAlias emp_cte
-            :        +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false
+            :        +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false, false
             +- SubqueryAlias d
                +- SubqueryAlias dept
                   +- View (`DEPT`, [dept_id#x, dept_name#x, state#x])
@@ -322,7 +322,7 @@
       :     +- Filter (count(1)#xL > cast(1 as bigint))
       :        +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL]
       :           +- SubqueryAlias empdept
-      :              +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false
+      :              +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false, false
       +- SubqueryAlias bonus
          +- View (`BONUS`, [emp_name#x, bonus_amt#x])
             +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x]
@@ -375,7 +375,7 @@
       :     +- Filter (count(1)#xL < cast(1 as bigint))
       :        +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL]
       :           +- SubqueryAlias empdept
-      :              +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false
+      :              +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false, false
       +- SubqueryAlias bonus
          +- View (`BONUS`, [emp_name#x, bonus_amt#x])
             +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out
index 230ffc0..5b50d64 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out
@@ -330,7 +330,7 @@
       +- Project [t1a#x, t1b#x, t1a#x, t1b#x]
          +- Join Inner, (t1b#x = t1b#x)
             :- SubqueryAlias cte1
-            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
             +- SubqueryAlias cte2
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out
index 199b876..9b547f4 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out
@@ -138,7 +138,7 @@
       :  +- Project [t1b#x]
       :     +- Filter (cast(t1b#x as int) > 0)
       :        +- SubqueryAlias cte1
-      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
       +- SubqueryAlias t1
          +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x])
             +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x]
@@ -197,21 +197,21 @@
          :        :  :     :- Project [t1b#x]
          :        :  :     :  +- Filter (cast(t1b#x as int) > 0)
          :        :  :     :     +- SubqueryAlias cte1
-         :        :  :     :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+         :        :  :     :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
          :        :  :     +- Project [t1b#x]
          :        :  :        +- Filter (cast(t1b#x as int) > 5)
          :        :  :           +- SubqueryAlias cte1
-         :        :  :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+         :        :  :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
          :        :  +- Intersect false
          :        :     :- Project [t1b#x]
          :        :     :  +- SubqueryAlias cte1
-         :        :     :     +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+         :        :     :     +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
          :        :     +- Project [t1b#x]
          :        :        +- SubqueryAlias cte1
-         :        :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+         :        :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
          :        +- Project [t1b#x]
          :           +- SubqueryAlias cte1
-         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
          +- SubqueryAlias t1
             +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x])
                +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x]
@@ -268,22 +268,22 @@
       :        :  :  :- Join FullOuter, (t1c#x = t1c#x)
       :        :  :  :  :- Join Inner, (t1b#x > t1b#x)
       :        :  :  :  :  :- SubqueryAlias cte1
-      :        :  :  :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false
+      :        :  :  :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false
       :        :  :  :  :  +- SubqueryAlias cte2
       :        :  :  :  :     +- SubqueryAlias cte1
-      :        :  :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false
+      :        :  :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false
       :        :  :  :  +- SubqueryAlias cte3
       :        :  :  :     +- SubqueryAlias cte1
-      :        :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false
+      :        :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false
       :        :  :  +- SubqueryAlias cte4
       :        :  :     +- SubqueryAlias cte1
-      :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false
+      :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false
       :        :  +- SubqueryAlias cte5
       :        :     +- SubqueryAlias cte1
-      :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false
+      :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false
       :        +- SubqueryAlias cte6
       :           +- SubqueryAlias cte1
-      :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false
+      :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false, false
       +- SubqueryAlias t1
          +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x])
             +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x]
@@ -354,16 +354,16 @@
             :- Join FullOuter, (t1a#x = t1a#x)
             :  :- Join Inner, ((cast(t1b#x as int) > 5) AND (t1a#x = t1a#x))
             :  :  :- SubqueryAlias cte1
-            :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+            :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
             :  :  +- SubqueryAlias cte2
             :  :     +- SubqueryAlias cte1
-            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
             :  +- SubqueryAlias cte3
             :     +- SubqueryAlias cte1
-            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
             +- SubqueryAlias cte4
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
 
 
 -- !query
@@ -424,10 +424,10 @@
          +- Project [t1a#x, t1b#x]
             +- Join Inner, (t1h#x >= t1h#x)
                :- SubqueryAlias cte1
-               :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false
+               :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false, false
                +- SubqueryAlias cte2
                   +- SubqueryAlias cte1
-                     +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false
+                     +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false, false
 
 
 -- !query
@@ -485,16 +485,16 @@
             :- Join RightOuter, (t1b#x = t1b#x)
             :  :- Join Inner, (t1a#x = t1a#x)
             :  :  :- SubqueryAlias cte1
-            :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false
+            :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false
             :  :  +- SubqueryAlias cte2
             :  :     +- SubqueryAlias cte1
-            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false
+            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false
             :  +- SubqueryAlias cte3
             :     +- SubqueryAlias cte1
-            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false
+            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false
             +- SubqueryAlias cte4
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false, false
 
 
 -- !query
@@ -538,10 +538,10 @@
       +- Project [t1a#x, t1b#x]
          +- Join RightOuter, (t1a#x = t1a#x)
             :- SubqueryAlias cte1
-            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
             +- SubqueryAlias cte2
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
 
 
 -- !query
@@ -599,15 +599,15 @@
          :        :           +- SubqueryAlias t1
          :        :              +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]
          :        +- SubqueryAlias cte1
-         :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+         :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
          +- SubqueryAlias s
             +- Project [t1b#x]
                +- Join LeftOuter, (t1b#x = t1b#x)
                   :- SubqueryAlias cte1
-                  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+                  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
                   +- SubqueryAlias cte2
                      +- SubqueryAlias cte1
-                        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+                        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
 
 
 -- !query
@@ -642,7 +642,7 @@
       :  +- Project [t1b#x]
       :     +- Filter (cast(t1b#x as int) < 0)
       :        +- SubqueryAlias cte1
-      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false
+      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false, false
       +- SubqueryAlias t1
          +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x])
             +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x]
@@ -722,16 +722,16 @@
          :        :- Join RightOuter, (t1b#x = t1b#x)
          :        :  :- Join Inner, (t1a#x = t1a#x)
          :        :  :  :- SubqueryAlias cte1
-         :        :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false
+         :        :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false
          :        :  :  +- SubqueryAlias cte2
          :        :  :     +- SubqueryAlias cte1
-         :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false
+         :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false
          :        :  +- SubqueryAlias cte3
          :        :     +- SubqueryAlias cte1
-         :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false
+         :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false
          :        +- SubqueryAlias cte4
          :           +- SubqueryAlias cte1
-         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false
+         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false, false
          +- SubqueryAlias t1
             +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x])
                +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out
index f64b373..9d6a817 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out
@@ -623,7 +623,7 @@
 :     :        +- OneRowRelation
 :     +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [a#x], false, false, 1
+:           +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
 +- SubqueryAlias t1
    +- View (`t1`, [c1#x, c2#x])
       +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x]
@@ -647,7 +647,7 @@
 :     :                       +- LocalRelation [c1#x, c2#x]
 :     +- Aggregate [sum(c2#x) AS sum(c2)#xL]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+:           +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
 +- SubqueryAlias t1
    +- View (`t1`, [c1#x, c2#x])
       +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x]
@@ -677,10 +677,10 @@
 :     :     +- Project [c1#x, c2#x]
 :     :        +- Filter (outer(c1#x) = c1#x)
 :     :           +- SubqueryAlias t3
-:     :              +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+:     :              +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
 :     +- Aggregate [sum(c2#x) AS sum(c2)#xL]
 :        +- SubqueryAlias t4
-:           +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+:           +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
 +- SubqueryAlias t1
    +- View (`t1`, [c1#x, c2#x])
       +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x]
@@ -713,10 +713,10 @@
 :                    +- Union false, false
 :                       :- Project [c1#x, c2#x]
 :                       :  +- SubqueryAlias t
-:                       :     +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+:                       :     +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
 :                       +- Project [c2#x, c1#x]
 :                          +- SubqueryAlias t
-:                             +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+:                             +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
 +- SubqueryAlias t1
    +- View (`t1`, [c1#x, c2#x])
       +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x]
@@ -756,9 +756,9 @@
       :           :  +- Aggregate [sum(c2#x) AS sum(c2)#xL]
       :           :     +- Filter (c1#x = outer(c1#x))
       :           :        +- SubqueryAlias t
-      :           :           +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+      :           :           +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
       :           +- SubqueryAlias v
-      :              +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false
+      :              +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, false
       +- SubqueryAlias t1
          +- View (`t1`, [c1#x, c2#x])
             +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x]
@@ -779,7 +779,7 @@
    :  +- Project [a#x]
    :     +- Filter (a#x = outer(c1#x))
    :        +- SubqueryAlias t
-   :           +- CTERelationRef xxxx, true, [a#x], false, false, 1
+   :           +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
    +- SubqueryAlias t1
       +- View (`t1`, [c1#x, c2#x])
          +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x]
@@ -1027,7 +1027,7 @@
    :  +- Aggregate [sum(1) AS sum(1)#xL]
    :     +- Filter ((a#x = cast(outer(col#x) as int)) OR (upper(cast(outer(col#x) as string)) = Y))
    :        +- SubqueryAlias T
-   :           +- CTERelationRef xxxx, true, [a#x], false, false, 1
+   :           +- CTERelationRef xxxx, true, [a#x], false, false, 1, false
    +- SubqueryAlias foo
       +- Project [null AS col#x]
          +- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out
index d537691..681b537 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out
@@ -888,10 +888,10 @@
    +- Join Inner, (b#x = b#x)
       :- SubqueryAlias t1
       :  +- SubqueryAlias temp
-      :     +- CTERelationRef xxxx, true, [b#x], false, false
+      :     +- CTERelationRef xxxx, true, [b#x], false, false, false
       +- SubqueryAlias t2
          +- SubqueryAlias temp
-            +- CTERelationRef xxxx, true, [b#x], false, false
+            +- CTERelationRef xxxx, true, [b#x], false, false, false
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out
index 3b04886..51bf54e 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out
@@ -833,6 +833,6 @@
          +- Project [coalesce(key#x, key#x) AS key#x, key#x]
             +- Join FullOuter, (key#x = key#x)
                :- SubqueryAlias t1
-               :  +- CTERelationRef xxxx, true, [key#x], false, false, 1
+               :  +- CTERelationRef xxxx, true, [key#x], false, false, 1, false
                +- SubqueryAlias t2
-                  +- CTERelationRef xxxx, true, [key#x], false, false, 1
+                  +- CTERelationRef xxxx, true, [key#x], false, false, 1, false
diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql
index 828a7c8..4aa7be7 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/cte-recursion.sql
@@ -97,6 +97,37 @@
 
 DROP VIEW ZeroAndOne;
 
+-- limited recursion allowed to stop from failing by putting LIMIT ALL
+WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS (
+    SELECT 1
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+SELECT * FROM t LIMIT ALL;
+
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+SELECT * FROM t LIMIT ALL;
+
+-- One reference is limit all but other isn't. Should fail.
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+   (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t);
+
+-- One references are limit all.
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+   (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t LIMIT ALL);
+
 -- terminate recursion with LIMIT
 WITH RECURSIVE r(level) AS (
   VALUES 0
diff --git a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out
index f1fea1f..536fc6c 100644
--- a/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/cte-recursion.sql.out
@@ -334,6 +334,302 @@
 
 
 -- !query
+WITH RECURSIVE t(n) MAX RECURSION LEVEL 100 AS (
+    SELECT 1
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+SELECT * FROM t LIMIT ALL
+-- !query schema
+struct<n:int>
+-- !query output
+1
+10
+11
+12
+13
+14
+15
+16
+17
+18
+19
+2
+20
+21
+22
+23
+24
+25
+26
+27
+28
+29
+3
+30
+31
+32
+33
+34
+35
+36
+37
+38
+39
+4
+40
+41
+42
+43
+44
+45
+46
+47
+48
+49
+5
+50
+51
+52
+53
+54
+55
+56
+57
+58
+59
+6
+60
+7
+8
+9
+
+
+-- !query
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+SELECT * FROM t LIMIT ALL
+-- !query schema
+struct<n:int>
+-- !query output
+1
+10
+11
+12
+13
+14
+15
+16
+17
+18
+19
+2
+20
+21
+22
+23
+24
+25
+26
+27
+28
+29
+3
+30
+31
+32
+33
+34
+35
+36
+37
+38
+39
+4
+40
+41
+42
+43
+44
+45
+46
+47
+48
+49
+5
+50
+51
+52
+53
+54
+55
+56
+57
+58
+59
+6
+60
+7
+8
+9
+
+
+-- !query
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+   (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkException
+{
+  "errorClass" : "RECURSION_ROW_LIMIT_EXCEEDED",
+  "sqlState" : "42836",
+  "messageParameters" : {
+    "rowLimit" : "50"
+  }
+}
+
+
+-- !query
+WITH RECURSIVE t MAX RECURSION LEVEL 100 AS (
+    SELECT 1 AS n
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 60
+    )
+   (SELECT n FROM t LIMIT ALL) UNION ALL (SELECT n FROM t LIMIT ALL)
+-- !query schema
+struct<n:int>
+-- !query output
+1
+1
+10
+10
+11
+11
+12
+12
+13
+13
+14
+14
+15
+15
+16
+16
+17
+17
+18
+18
+19
+19
+2
+2
+20
+20
+21
+21
+22
+22
+23
+23
+24
+24
+25
+25
+26
+26
+27
+27
+28
+28
+29
+29
+3
+3
+30
+30
+31
+31
+32
+32
+33
+33
+34
+34
+35
+35
+36
+36
+37
+37
+38
+38
+39
+39
+4
+4
+40
+40
+41
+41
+42
+42
+43
+43
+44
+44
+45
+45
+46
+46
+47
+47
+48
+48
+49
+49
+5
+5
+50
+50
+51
+51
+52
+52
+53
+53
+54
+54
+55
+55
+56
+56
+57
+57
+58
+58
+59
+59
+6
+6
+60
+60
+7
+7
+8
+8
+9
+9
+
+
+-- !query
 WITH RECURSIVE r(level) AS (
   VALUES 0
   UNION ALL