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

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


The following commit(s) were added to refs/heads/master by this push:
     new 43046631a5d [SPARK-46062][SQL] Sync the isStreaming flag between CTE 
definition and reference
43046631a5d is described below

commit 43046631a5d4ac7201361a00473cc87fa52ab5a7
Author: Jungtaek Lim <kabhwan.opensou...@gmail.com>
AuthorDate: Thu Nov 23 22:32:16 2023 +0900

    [SPARK-46062][SQL] Sync the isStreaming flag between CTE definition and 
reference
    
    ### What changes were proposed in this pull request?
    
    This PR proposes to sync the flag `isStreaming` from CTE definition to CTE 
reference.
    
    The essential issue is that CTE reference node cannot determine the flag 
`isStreaming` by itself, and never be able to have a proper value and always 
takes the default as it does not have a parameter in constructor. The other 
flag `resolved` is handled, and we need to do the same for `isStreaming`.
    
    Once we add the parameter to the constructor, we will also need to make 
sure the flag is in sync with CTE definition. We have a rule `ResolveWithCTE` 
doing the sync, hence we add the logic to sync the flag `isStreaming` as well.
    
    ### Why are the changes needed?
    
    The bug may impact some rules which behaves differently depending on 
isStreaming flag. It would no longer be a problem once CTE reference is 
replaced with CTE definition at some point in "optimization phase", but all 
rules in analyzer and optimizer being triggered before the rule takes effect 
may misbehave based on incorrect isStreaming flag.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    New UT.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #43966 from HeartSaVioR/SPARK-46062.
    
    Authored-by: Jungtaek Lim <kabhwan.opensou...@gmail.com>
    Signed-off-by: Jungtaek Lim <kabhwan.opensou...@gmail.com>
---
 .../sql/catalyst/analysis/CTESubstitution.scala    |  2 +-
 .../sql/catalyst/analysis/ResolveWithCTE.scala     |  2 +-
 .../catalyst/optimizer/MergeScalarSubqueries.scala |  3 +-
 ...ushdownPredicatesAndPruneColumnsForCTEDef.scala |  2 +-
 .../plans/logical/basicLogicalOperators.scala      |  1 +
 .../sql/catalyst/analysis/AnalysisSuite.scala      | 27 +++++++--
 .../optimizer/MergeScalarSubqueriesSuite.scala     |  3 +-
 .../ansi/double-quoted-identifiers-enabled.sql.out |  4 +-
 .../sql-tests/analyzer-results/cte-command.sql.out | 10 ++--
 .../sql-tests/analyzer-results/cte-nested.sql.out  | 38 ++++++------
 .../analyzer-results/cte-nonlegacy.sql.out         | 64 ++++++++++----------
 .../sql-tests/analyzer-results/cte.sql.out         | 70 +++++++++++-----------
 .../analyzer-results/join-lateral.sql.out          |  4 +-
 .../analyzer-results/non-excludable-rule.sql.out   |  2 +-
 .../postgreSQL/window_part3.sql.out                |  8 +--
 .../analyzer-results/postgreSQL/with.sql.out       | 14 ++---
 .../analyzer-results/sql-session-variables.sql.out |  2 +-
 .../subquery/exists-subquery/exists-cte.sql.out    | 16 ++---
 .../in-subquery/in-multiple-columns.sql.out        |  4 +-
 .../subquery/in-subquery/in-with-cte.sql.out       | 64 ++++++++++----------
 .../scalar-subquery/scalar-subquery-select.sql.out | 20 +++----
 .../sql-tests/analyzer-results/transform.sql.out   |  4 +-
 .../sql-tests/analyzer-results/using-join.sql.out  |  4 +-
 .../spark/sql/streaming/StreamingQuerySuite.scala  | 47 ++++++++++++++-
 24 files changed, 239 insertions(+), 176 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala
index e98cf5c330e..2982d8477fc 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala
@@ -283,7 +283,7 @@ object CTESubstitution extends Rule[LogicalPlan] {
             d.child
           } else {
             // Add a `SubqueryAlias` for hint-resolving rules to match 
relation names.
-            SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output))
+            SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, 
d.isStreaming))
           }
         }.getOrElse(u)
 
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 78b776f12f0..f1077378b2d 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
@@ -51,7 +51,7 @@ object ResolveWithCTE extends Rule[LogicalPlan] {
 
       case ref: CTERelationRef if !ref.resolved =>
         cteDefMap.get(ref.cteId).map { cteDef =>
-          CTERelationRef(cteDef.id, cteDef.resolved, cteDef.output)
+          CTERelationRef(cteDef.id, cteDef.resolved, cteDef.output, 
cteDef.isStreaming)
         }.getOrElse {
           ref
         }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
index 6184160829b..ff0bc5e66d7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
@@ -381,7 +381,8 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] {
               val subqueryCTE = header.plan.asInstanceOf[CTERelationDef]
               GetStructField(
                 ScalarSubquery(
-                  CTERelationRef(subqueryCTE.id, _resolved = true, 
subqueryCTE.output),
+                  CTERelationRef(subqueryCTE.id, _resolved = true, 
subqueryCTE.output,
+                    subqueryCTE.isStreaming),
                   exprId = ssr.exprId),
                 ssr.headerIndex)
             } else {
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 e643a1af363..aa13e6a67c5 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 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends 
Rule[LogicalPlan] {
         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/plans/logical/basicLogicalOperators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index 05abaf30906..ff59e60482d 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
@@ -862,6 +862,7 @@ case class CTERelationRef(
     cteId: Long,
     _resolved: Boolean,
     override val output: Seq[Attribute],
+    override val isStreaming: Boolean,
     statsOpt: Option[Statistics] = None) extends LeafNode with 
MultiInstanceRelation {
 
   final override val nodePatterns: Seq[TreePattern] = Seq(CTE)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index d035dcad81e..b86d82a7f24 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -1573,7 +1573,7 @@ class AnalysisSuite extends AnalysisTest with Matchers {
   test("SPARK-43030: deduplicate relations in CTE relation definitions") {
     val join = testRelation.as("left").join(testRelation.as("right"))
     val cteDef = CTERelationDef(join)
-    val cteRef = CTERelationRef(cteDef.id, false, Nil)
+    val cteRef = CTERelationRef(cteDef.id, false, Nil, false)
 
     withClue("flat CTE") {
       val plan = WithCTE(cteRef.select($"left.a"), Seq(cteDef)).analyze
@@ -1586,7 +1586,7 @@ class AnalysisSuite extends AnalysisTest with Matchers {
 
     withClue("nested CTE") {
       val cteDef2 = CTERelationDef(WithCTE(cteRef.join(testRelation), 
Seq(cteDef)))
-      val cteRef2 = CTERelationRef(cteDef2.id, false, Nil)
+      val cteRef2 = CTERelationRef(cteDef2.id, false, Nil, false)
       val plan = WithCTE(cteRef2, Seq(cteDef2)).analyze
       val relations = plan.collect {
         case r: LocalRelation => r
@@ -1598,7 +1598,7 @@ class AnalysisSuite extends AnalysisTest with Matchers {
 
   test("SPARK-43030: deduplicate CTE relation references") {
     val cteDef = CTERelationDef(testRelation.select($"a"))
-    val cteRef = CTERelationRef(cteDef.id, false, Nil)
+    val cteRef = CTERelationRef(cteDef.id, false, Nil, false)
 
     withClue("single reference") {
       val plan = WithCTE(cteRef.where($"a" > 1), Seq(cteDef)).analyze
@@ -1621,7 +1621,7 @@ class AnalysisSuite extends AnalysisTest with Matchers {
 
     withClue("CTE relation has duplicated attributes") {
       val cteDef = CTERelationDef(testRelation.select($"a", $"a"))
-      val cteRef = CTERelationRef(cteDef.id, false, Nil)
+      val cteRef = CTERelationRef(cteDef.id, false, Nil, false)
       val plan = WithCTE(cteRef.join(cteRef.select($"a")), Seq(cteDef)).analyze
       val refs = plan.collect {
         case r: CTERelationRef => r
@@ -1633,14 +1633,14 @@ class AnalysisSuite extends AnalysisTest with Matchers {
     withClue("CTE relation has duplicate aliases") {
       val alias = Alias($"a", "x")()
       val cteDef = CTERelationDef(testRelation.select(alias, alias).where($"x" 
=== 1))
-      val cteRef = CTERelationRef(cteDef.id, false, Nil)
+      val cteRef = CTERelationRef(cteDef.id, false, Nil, false)
       // Should not fail with the assertion failure: Found duplicate rewrite 
attributes.
       WithCTE(cteRef.join(cteRef), Seq(cteDef)).analyze
     }
 
     withClue("references in both CTE relation definition and main query") {
       val cteDef2 = CTERelationDef(cteRef.where($"a" > 2))
-      val cteRef2 = CTERelationRef(cteDef2.id, false, Nil)
+      val cteRef2 = CTERelationRef(cteDef2.id, false, Nil, false)
       val plan = WithCTE(cteRef.union(cteRef2), Seq(cteDef, cteDef2)).analyze
       val refs = plan.collect {
         case r: CTERelationRef => r
@@ -1747,4 +1747,19 @@ class AnalysisSuite extends AnalysisTest with Matchers {
     // EventTimeWatermark node is NOT eliminated.
     assert(analyzed.exists(_.isInstanceOf[EventTimeWatermark]))
   }
+
+  test("SPARK-46062: isStreaming flag is synced from CTE definition to CTE 
reference") {
+    val cteDef = CTERelationDef(streamingRelation.select($"a", $"ts"))
+    // Intentionally marking the flag _resolved to false, so that analyzer has 
a chance to sync
+    // the flag isStreaming on syncing the flag _resolved.
+    val cteRef = CTERelationRef(cteDef.id, _resolved = false, Nil, isStreaming 
= false)
+    val plan = WithCTE(cteRef, Seq(cteDef)).analyze
+
+    val refs = plan.collect {
+      case r: CTERelationRef => r
+    }
+    assert(refs.length == 1)
+    assert(refs.head.resolved)
+    assert(refs.head.isStreaming)
+  }
 }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala
index 941be5f0c4e..3f7fad27e7b 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala
@@ -42,7 +42,8 @@ class MergeScalarSubqueriesSuite extends PlanTest {
   }
 
   private def extractorExpression(cteIndex: Int, output: Seq[Attribute], 
fieldIndex: Int) = {
-    GetStructField(ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, 
output)), fieldIndex)
+    GetStructField(ScalarSubquery(
+      CTERelationRef(cteIndex, _resolved = true, output, isStreaming = 
false)), fieldIndex)
       .as("scalarsubquery()")
   }
 
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out
 
b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out
index d7636445653..c37acb7879c 100644
--- 
a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out
@@ -418,7 +418,7 @@ CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS 
(SELECT 1) SELECT "a"
       :           +- OneRowRelation
       +- Project [a#x]
          +- SubqueryAlias v
-            +- CTERelationRef xxxx, true, [a#x]
+            +- CTERelationRef xxxx, true, [a#x], false
 
 
 -- !query
@@ -438,7 +438,7 @@ Project [a1#x AS a2#x]
                   :           +- OneRowRelation
                   +- Project [a#x]
                      +- SubqueryAlias v
-                        +- CTERelationRef xxxx, true, [a#x]
+                        +- CTERelationRef xxxx, true, [a#x], false
 
 
 -- !query
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 93dab5aaa6d..4aac75ec45a 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 @@ CreateDataSourceTableAsSelectCommand 
`spark_catalog`.`default`.`cte_tbl`, ErrorI
       :        +- OneRowRelation
       +- Project [col#x]
          +- SubqueryAlias s
-            +- CTERelationRef xxxx, true, [col#x]
+            +- CTERelationRef xxxx, true, [col#x], false
 
 
 -- !query
@@ -32,7 +32,7 @@ CreateViewCommand `cte_view`, WITH s AS (SELECT 42 AS col) 
SELECT * FROM s, fals
       :        +- OneRowRelation
       +- Project [col#x]
          +- SubqueryAlias s
-            +- CTERelationRef xxxx, true, [col#x]
+            +- CTERelationRef xxxx, true, [col#x], false
 
 
 -- !query
@@ -49,7 +49,7 @@ Project [col#x]
             :        +- OneRowRelation
             +- Project [col#x]
                +- SubqueryAlias s
-                  +- CTERelationRef xxxx, true, [col#x]
+                  +- CTERelationRef xxxx, true, [col#x], false
 
 
 -- !query
@@ -64,7 +64,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in 
comparison]/{warehouse_d
    :        +- OneRowRelation
    +- Project [col#x]
       +- SubqueryAlias S
-         +- CTERelationRef xxxx, true, [col#x]
+         +- CTERelationRef xxxx, true, [col#x], false
 
 
 -- !query
@@ -86,7 +86,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in 
comparison]/{warehouse_d
    :        +- OneRowRelation
    +- Project [col#x]
       +- SubqueryAlias s
-         +- CTERelationRef xxxx, true, [col#x]
+         +- CTERelationRef xxxx, true, [col#x], 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 2238ced28ee..6e55c6fa83c 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 @@ WithCTE
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x]
+      +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -37,7 +37,7 @@ Aggregate [max(c#x) AS max(c)#x]
       :           +- OneRowRelation
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x]
+            +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -54,7 +54,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x]
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- OneRowRelation
 
 
@@ -137,11 +137,11 @@ WithCTE
 :        :           :           +- OneRowRelation
 :        :           +- Project [c#x]
 :        :              +- SubqueryAlias t
-:        :                 +- CTERelationRef xxxx, true, [c#x]
+:        :                 +- CTERelationRef xxxx, true, [c#x], false
 :        +- OneRowRelation
 +- Project [scalarsubquery()#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [scalarsubquery()#x]
+      +- CTERelationRef xxxx, true, [scalarsubquery()#x], false
 
 
 -- !query
@@ -191,7 +191,7 @@ WithCTE
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x]
+            +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -220,7 +220,7 @@ WithCTE
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x]
+                  +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -255,7 +255,7 @@ WithCTE
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x]
+                  +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -358,14 +358,14 @@ WithCTE
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [2 AS 2#x]
 :        +- OneRowRelation
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x]
+      +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -428,15 +428,15 @@ WithCTE
 :  +- SubqueryAlias t3
 :     +- Project [1#x]
 :        +- SubqueryAlias t1
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [1#x]
 :        +- SubqueryAlias t3
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [1#x]
+      +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -459,12 +459,12 @@ WithCTE
 :  +- SubqueryAlias cte_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x]
+            +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -492,19 +492,19 @@ WithCTE
 :  +- SubqueryAlias cte_inner_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], 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]
+:                 +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x]
+            +- CTERelationRef xxxx, true, [1#x], 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 bd9b443d01d..f1a302b06f2 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 @@ WithCTE
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x]
+      +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -37,7 +37,7 @@ Aggregate [max(c#x) AS max(c)#x]
       :           +- OneRowRelation
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x]
+            +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -54,7 +54,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x]
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- OneRowRelation
 
 
@@ -106,10 +106,10 @@ WithCTE
 :  +- SubqueryAlias t2
 :     +- Project [2#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [2#x]
+:           +- CTERelationRef xxxx, true, [2#x], false
 +- Project [2#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [2#x]
+      +- CTERelationRef xxxx, true, [2#x], false
 
 
 -- !query
@@ -144,11 +144,11 @@ WithCTE
 :        :           :           +- OneRowRelation
 :        :           +- Project [c#x]
 :        :              +- SubqueryAlias t
-:        :                 +- CTERelationRef xxxx, true, [c#x]
+:        :                 +- CTERelationRef xxxx, true, [c#x], false
 :        +- OneRowRelation
 +- Project [scalarsubquery()#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [scalarsubquery()#x]
+      +- CTERelationRef xxxx, true, [scalarsubquery()#x], false
 
 
 -- !query
@@ -181,15 +181,15 @@ WithCTE
 :  +- SubqueryAlias t2
 :     +- Project [3#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [3#x]
+:           +- CTERelationRef xxxx, true, [3#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [3#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [3#x]
+:           +- CTERelationRef xxxx, true, [3#x], false
 +- Project [3#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [3#x]
+      +- CTERelationRef xxxx, true, [3#x], false
 
 
 -- !query
@@ -214,7 +214,7 @@ WithCTE
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [c#x]
          +- SubqueryAlias t
-            +- CTERelationRef xxxx, true, [c#x]
+            +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -243,7 +243,7 @@ WithCTE
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x]
+                  +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -278,7 +278,7 @@ WithCTE
          +- SubqueryAlias __auto_generated_subquery_name
             +- Project [c#x]
                +- SubqueryAlias t
-                  +- CTERelationRef xxxx, true, [c#x]
+                  +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -301,7 +301,7 @@ WithCTE
    :     :        +- OneRowRelation
    :     +- Project [2#x]
    :        +- SubqueryAlias t
-   :           +- CTERelationRef xxxx, true, [2#x]
+   :           +- CTERelationRef xxxx, true, [2#x], false
    +- OneRowRelation
 
 
@@ -328,7 +328,7 @@ WithCTE
    :     :     :        +- OneRowRelation
    :     :     +- Project [2#x]
    :     :        +- SubqueryAlias t
-   :     :           +- CTERelationRef xxxx, true, [2#x]
+   :     :           +- CTERelationRef xxxx, true, [2#x], false
    :     +- OneRowRelation
    +- OneRowRelation
 
@@ -362,7 +362,7 @@ WithCTE
    :        :     :        +- OneRowRelation
    :        :     +- Project [3#x]
    :        :        +- SubqueryAlias t
-   :        :           +- CTERelationRef xxxx, true, [3#x]
+   :        :           +- CTERelationRef xxxx, true, [3#x], false
    :        +- OneRowRelation
    +- OneRowRelation
 
@@ -391,9 +391,9 @@ WithCTE
       :     :           +- OneRowRelation
       :     +- Project [c#x]
       :        +- SubqueryAlias t
-      :           +- CTERelationRef xxxx, true, [c#x]
+      :           +- CTERelationRef xxxx, true, [c#x], false
       +- SubqueryAlias t
-         +- CTERelationRef xxxx, true, [c#x]
+         +- CTERelationRef xxxx, true, [c#x], false
 
 
 -- !query
@@ -414,14 +414,14 @@ WithCTE
 :  +- SubqueryAlias t
 :     +- Project [1#x]
 :        +- SubqueryAlias t2
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [2 AS 2#x]
 :        +- OneRowRelation
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x]
+      +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -446,10 +446,10 @@ WithCTE
 :  +- SubqueryAlias t
 :     +- Project [2#x]
 :        +- SubqueryAlias aBC
-:           +- CTERelationRef xxxx, true, [2#x]
+:           +- CTERelationRef xxxx, true, [2#x], false
 +- Project [2#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [2#x]
+      +- CTERelationRef xxxx, true, [2#x], false
 
 
 -- !query
@@ -472,7 +472,7 @@ WithCTE
    :     :        +- OneRowRelation
    :     +- Project [2#x]
    :        +- SubqueryAlias aBC
-   :           +- CTERelationRef xxxx, true, [2#x]
+   :           +- CTERelationRef xxxx, true, [2#x], false
    +- OneRowRelation
 
 
@@ -496,15 +496,15 @@ WithCTE
 :  +- SubqueryAlias t3
 :     +- Project [1#x]
 :        +- SubqueryAlias t1
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias t2
 :     +- Project [1#x]
 :        +- SubqueryAlias t3
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias t2
-      +- CTERelationRef xxxx, true, [1#x]
+      +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -527,12 +527,12 @@ WithCTE
 :  +- SubqueryAlias cte_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x]
+            +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -560,19 +560,19 @@ WithCTE
 :  +- SubqueryAlias cte_inner_inner
 :     +- Project [1#x]
 :        +- SubqueryAlias cte_outer
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], 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]
+:                 +- CTERelationRef xxxx, true, [1#x], false
 +- Project [1#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [1#x]
          +- SubqueryAlias cte_inner
-            +- CTERelationRef xxxx, true, [1#x]
+            +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
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 e06020ac06c..1d9d812875c 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
@@ -73,7 +73,7 @@ WithCTE
 :                       +- LocalRelation [id#x]
 +- Project [1#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [1#x]
+      +- CTERelationRef xxxx, true, [1#x], false
 
 
 -- !query
@@ -113,13 +113,13 @@ WithCTE
 :  +- SubqueryAlias t2
 :     +- Project [2 AS 2#x]
 :        +- SubqueryAlias t1
-:           +- CTERelationRef xxxx, true, [id#x]
+:           +- CTERelationRef xxxx, true, [id#x], false
 +- Project [id#x, 2#x]
    +- Join Cross
       :- SubqueryAlias t1
-      :  +- CTERelationRef xxxx, true, [id#x]
+      :  +- CTERelationRef xxxx, true, [id#x], false
       +- SubqueryAlias t2
-         +- CTERelationRef xxxx, true, [2#x]
+         +- CTERelationRef xxxx, true, [2#x], false
 
 
 -- !query
@@ -157,10 +157,10 @@ WithCTE
    +- Join Cross
       :- SubqueryAlias t1
       :  +- SubqueryAlias CTE1
-      :     +- CTERelationRef xxxx, true, [id#x]
+      :     +- CTERelationRef xxxx, true, [id#x], false
       +- SubqueryAlias t2
          +- SubqueryAlias CTE1
-            +- CTERelationRef xxxx, true, [id#x]
+            +- CTERelationRef xxxx, true, [id#x], false
 
 
 -- !query
@@ -176,7 +176,7 @@ WithCTE
 +- Project [x#x]
    +- Filter (x#x = 1)
       +- SubqueryAlias t
-         +- CTERelationRef xxxx, true, [x#x]
+         +- CTERelationRef xxxx, true, [x#x], false
 
 
 -- !query
@@ -192,7 +192,7 @@ WithCTE
 +- Project [x#x, y#x]
    +- Filter ((x#x = 1) AND (y#x = 2))
       +- SubqueryAlias t
-         +- CTERelationRef xxxx, true, [x#x, y#x]
+         +- CTERelationRef xxxx, true, [x#x, y#x], false
 
 
 -- !query
@@ -207,7 +207,7 @@ WithCTE
 :           +- OneRowRelation
 +- Project [x#x, x#x]
    +- SubqueryAlias t
-      +- CTERelationRef xxxx, true, [x#x, x#x]
+      +- CTERelationRef xxxx, true, [x#x, x#x], false
 
 
 -- !query
@@ -309,46 +309,46 @@ WithCTE
 :     +- Project [c8#x AS c7#x]
 :        +- Project [c8#x]
 :           +- SubqueryAlias w8
-:              +- CTERelationRef xxxx, true, [c8#x]
+:              +- CTERelationRef xxxx, true, [c8#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w6
 :     +- Project [c7#x AS c6#x]
 :        +- Project [c7#x]
 :           +- SubqueryAlias w7
-:              +- CTERelationRef xxxx, true, [c7#x]
+:              +- CTERelationRef xxxx, true, [c7#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w5
 :     +- Project [c6#x AS c5#x]
 :        +- Project [c6#x]
 :           +- SubqueryAlias w6
-:              +- CTERelationRef xxxx, true, [c6#x]
+:              +- CTERelationRef xxxx, true, [c6#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w4
 :     +- Project [c5#x AS c4#x]
 :        +- Project [c5#x]
 :           +- SubqueryAlias w5
-:              +- CTERelationRef xxxx, true, [c5#x]
+:              +- CTERelationRef xxxx, true, [c5#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w3
 :     +- Project [c4#x AS c3#x]
 :        +- Project [c4#x]
 :           +- SubqueryAlias w4
-:              +- CTERelationRef xxxx, true, [c4#x]
+:              +- CTERelationRef xxxx, true, [c4#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w2
 :     +- Project [c3#x AS c2#x]
 :        +- Project [c3#x]
 :           +- SubqueryAlias w3
-:              +- CTERelationRef xxxx, true, [c3#x]
+:              +- CTERelationRef xxxx, true, [c3#x], false
 :- CTERelationDef xxxx, false
 :  +- SubqueryAlias w1
 :     +- Project [c2#x AS c1#x]
 :        +- Project [c2#x]
 :           +- SubqueryAlias w2
-:              +- CTERelationRef xxxx, true, [c2#x]
+:              +- CTERelationRef xxxx, true, [c2#x], false
 +- Project [c1#x]
    +- SubqueryAlias w1
-      +- CTERelationRef xxxx, true, [c1#x]
+      +- CTERelationRef xxxx, true, [c1#x], false
 
 
 -- !query
@@ -385,7 +385,7 @@ WithCTE
 +- Project [42#x, 10#x]
    +- Join Inner
       :- SubqueryAlias same_name
-      :  +- CTERelationRef xxxx, true, [42#x]
+      :  +- CTERelationRef xxxx, true, [42#x], false
       +- SubqueryAlias same_name
          +- Project [10 AS 10#x]
             +- OneRowRelation
@@ -424,7 +424,7 @@ WithCTE
 :        +- OneRowRelation
 +- Project [x#x, typeof(x#x) AS typeof(x)#x]
    +- SubqueryAlias q
-      +- CTERelationRef xxxx, true, [x#x]
+      +- CTERelationRef xxxx, true, [x#x], false
 
 
 -- !query
@@ -484,7 +484,7 @@ Project [y#x]
       :        +- OneRowRelation
       +- Project [(x#x + 1) AS y#x]
          +- SubqueryAlias q
-            +- CTERelationRef xxxx, true, [x#x]
+            +- CTERelationRef xxxx, true, [x#x], false
 
 
 -- !query
@@ -498,7 +498,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x]
 :     :        +- OneRowRelation
 :     +- Project [x#x]
 :        +- SubqueryAlias q
-:           +- CTERelationRef xxxx, true, [x#x]
+:           +- CTERelationRef xxxx, true, [x#x], false
 +- OneRowRelation
 
 
@@ -513,7 +513,7 @@ Project [1 IN (list#x []) AS (1 IN (listquery()))#x]
 :     :        +- OneRowRelation
 :     +- Project [1#x]
 :        +- SubqueryAlias q
-:           +- CTERelationRef xxxx, true, [1#x]
+:           +- CTERelationRef xxxx, true, [1#x], false
 +- OneRowRelation
 
 
@@ -561,14 +561,14 @@ WithCTE
       :- Join Inner
       :  :- SubqueryAlias x
       :  :  +- SubqueryAlias T1
-      :  :     +- CTERelationRef xxxx, true, [a#x]
+      :  :     +- CTERelationRef xxxx, true, [a#x], false
       :  +- SubqueryAlias y
       :     +- Project [b#x]
       :        +- SubqueryAlias T1
-      :           +- CTERelationRef xxxx, true, [b#x]
+      :           +- CTERelationRef xxxx, true, [b#x], false
       +- SubqueryAlias z
          +- SubqueryAlias T1
-            +- CTERelationRef xxxx, true, [a#x]
+            +- CTERelationRef xxxx, true, [a#x], false
 
 
 -- !query
@@ -596,9 +596,9 @@ WithCTE
       +- Project [c#x, a#x]
          +- Join Inner
             :- SubqueryAlias ttTT
-            :  +- CTERelationRef xxxx, true, [c#x]
+            :  +- CTERelationRef xxxx, true, [c#x], false
             +- SubqueryAlias tttT_2
-               +- CTERelationRef xxxx, true, [a#x]
+               +- CTERelationRef xxxx, true, [a#x], false
 
 
 -- !query
@@ -614,7 +614,7 @@ Project [scalar-subquery#x [x#x] AS scalarsubquery(x)#x]
 :     :        +- OneRowRelation
 :     +- Project [x#x]
 :        +- SubqueryAlias q
-:           +- CTERelationRef xxxx, true, [x#x]
+:           +- CTERelationRef xxxx, true, [x#x], false
 +- SubqueryAlias T
    +- Project [1 AS x#x, 2 AS y#x]
       +- OneRowRelation
@@ -633,7 +633,7 @@ Project [scalar-subquery#x [x#x && y#x] AS 
scalarsubquery(x, y)#x]
 :     :        +- 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]
+:           +- CTERelationRef xxxx, true, [z#x], false
 +- SubqueryAlias T
    +- Project [1 AS x#x, 2 AS y#x]
       +- OneRowRelation
@@ -653,12 +653,12 @@ WithCTE
 :  +- SubqueryAlias q2
 :     +- Project [x#x]
 :        +- SubqueryAlias q1
-:           +- CTERelationRef xxxx, true, [x#x]
+:           +- CTERelationRef xxxx, true, [x#x], false
 +- Project [x#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [x#x]
          +- SubqueryAlias q2
-            +- CTERelationRef xxxx, true, [x#x]
+            +- CTERelationRef xxxx, true, [x#x], false
 
 
 -- !query
@@ -675,12 +675,12 @@ WithCTE
 :  +- SubqueryAlias q1
 :     +- Project [(x#x + 1) AS (x + 1)#x]
 :        +- SubqueryAlias q1
-:           +- CTERelationRef xxxx, true, [x#x]
+:           +- CTERelationRef xxxx, true, [x#x], false
 +- Project [(x + 1)#x]
    +- SubqueryAlias __auto_generated_subquery_name
       +- Project [(x + 1)#x]
          +- SubqueryAlias q1
-            +- CTERelationRef xxxx, true, [(x + 1)#x]
+            +- CTERelationRef xxxx, true, [(x + 1)#x], false
 
 
 -- !query
@@ -721,9 +721,9 @@ WithCTE
       :  +- Aggregate [max(j#x) AS max(j)#x]
       :     +- SubqueryAlias cte2
       :        +- SubqueryAlias cte1
-      :           +- CTERelationRef xxxx, true, [j#x]
+      :           +- CTERelationRef xxxx, true, [j#x], false
       +- SubqueryAlias cte1
-         +- CTERelationRef xxxx, true, [j#x]
+         +- CTERelationRef xxxx, true, [j#x], 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 09b32afd1f4..faa582035ef 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 @@ WithCTE
 :           :                 +- 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]
+:              +- CTERelationRef xxxx, true, [c1#x], false
 +- Project [c1#x, c2#x]
    +- SubqueryAlias cte2
-      +- CTERelationRef xxxx, true, [c1#x, c2#x]
+      +- CTERelationRef xxxx, true, [c1#x, c2#x], 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 305a59f01e4..b80bed6f7c2 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 @@ WithCTE
    +- Filter (id#xL > scalar-subquery#x [])
       :  +- Aggregate [max(id#xL) AS max(id)#xL]
       :     +- SubqueryAlias tmp
-      :        +- CTERelationRef xxxx, true, [id#xL]
+      :        +- CTERelationRef xxxx, true, [id#xL], false
       +- Range (0, 3, step=1, splits=None)
 
 
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 f0f111c6857..78a3c60d20d 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 @@ WithCTE
       +- 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]
+               +- CTERelationRef xxxx, true, [x#xL], false
 
 
 -- !query
@@ -121,7 +121,7 @@ WithCTE
       +- 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]
+               +- CTERelationRef xxxx, true, [x#xL], false
 
 
 -- !query
@@ -154,7 +154,7 @@ WithCTE
       +- 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]
+               +- CTERelationRef xxxx, true, [x#xL], false
 
 
 -- !query
@@ -187,7 +187,7 @@ WithCTE
       +- 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]
+               +- CTERelationRef xxxx, true, [x#xL], 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 3772b1dc8b2..f29b12d5af1 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 @@ WithCTE
 +- Project [x#x, y#x, x#x, y#x]
    +- Join Inner
       :- SubqueryAlias q1
-      :  +- CTERelationRef xxxx, true, [x#x, y#x]
+      :  +- CTERelationRef xxxx, true, [x#x, y#x], false
       +- SubqueryAlias q2
          +- SubqueryAlias q1
-            +- CTERelationRef xxxx, true, [x#x, y#x]
+            +- CTERelationRef xxxx, true, [x#x, y#x], false
 
 
 -- !query
@@ -194,7 +194,7 @@ WithCTE
    +- SubqueryAlias q
       +- Project [foo#x]
          +- SubqueryAlias cte
-            +- CTERelationRef xxxx, true, [foo#x]
+            +- CTERelationRef xxxx, true, [foo#x], false
 
 
 -- !query
@@ -222,13 +222,13 @@ WithCTE
 :                 +- Union false, false
 :                    :- Project [2#x]
 :                    :  +- SubqueryAlias innermost
-:                    :     +- CTERelationRef xxxx, true, [2#x]
+:                    :     +- CTERelationRef xxxx, true, [2#x], false
 :                    +- Project [3 AS 3#x]
 :                       +- OneRowRelation
 +- Sort [x#x ASC NULLS FIRST], true
    +- Project [x#x]
       +- SubqueryAlias outermost
-         +- CTERelationRef xxxx, true, [x#x]
+         +- CTERelationRef xxxx, true, [x#x], false
 
 
 -- !query
@@ -418,7 +418,7 @@ WithCTE
 :        +- OneRowRelation
 +- Project [x#x]
    +- SubqueryAlias ordinality
-      +- CTERelationRef xxxx, true, [x#x]
+      +- CTERelationRef xxxx, true, [x#x], false
 
 
 -- !query
@@ -459,7 +459,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in 
comparison]/{warehouse_d
       :        +- OneRowRelation
       +- Project [42#x]
          +- SubqueryAlias test
-            +- CTERelationRef xxxx, true, [42#x]
+            +- CTERelationRef xxxx, true, [42#x], 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 8d5251d3b2b..e75c7946ef7 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
@@ -2054,7 +2054,7 @@ WithCTE
 :        +- OneRowRelation
 +- Project [c1#x AS 1#x]
    +- SubqueryAlias v1
-      +- CTERelationRef xxxx, true, [c1#x]
+      +- CTERelationRef xxxx, true, [c1#x], false
 
 
 -- !query
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 2cd6ba53563..cab83b26499 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 @@ WithCTE
       :     +- Filter (outer(emp_name#x) = emp_name#x)
       :        +- SubqueryAlias b
       :           +- SubqueryAlias bonus_cte
-      :              +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x]
+      :              +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], 
false
       +- SubqueryAlias a
          +- SubqueryAlias bonus
             +- View (`BONUS`, [emp_name#x,bonus_amt#x])
@@ -189,10 +189,10 @@ WithCTE
       :        +- 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]
+      :           :     +- CTERelationRef xxxx, true, [id#x, emp_name#x, 
hiredate#x, salary#x, dept_id#x], false
       :           +- SubqueryAlias b
       :              +- SubqueryAlias dept_cte
-      :                 +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, 
state#x]
+      :                 +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, 
state#x], 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 @@ WithCTE
          :        +- 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]
+         :           :     +- CTERelationRef xxxx, true, [id#x, emp_name#x, 
hiredate#x, salary#x, dept_id#x], false
          :           +- SubqueryAlias b
          :              +- SubqueryAlias dept_cte
-         :                 +- CTERelationRef xxxx, true, [dept_id#x, 
dept_name#x, state#x]
+         :                 +- CTERelationRef xxxx, true, [dept_id#x, 
dept_name#x, state#x], false
          +- Join Inner
             :- Join Inner
             :  :- SubqueryAlias b
@@ -268,7 +268,7 @@ WithCTE
             :  :                 +- 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]
+            :        +- CTERelationRef xxxx, true, [id#x, emp_name#x, 
hiredate#x, salary#x, dept_id#x], false
             +- SubqueryAlias d
                +- SubqueryAlias dept
                   +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])
@@ -322,7 +322,7 @@ WithCTE
       :     +- 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]
+      :              +- CTERelationRef xxxx, true, [id#x, salary#x, 
emp_name#x, dept_id#x], 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 @@ WithCTE
       :     +- 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]
+      :              +- CTERelationRef xxxx, true, [id#x, salary#x, 
emp_name#x, dept_id#x], 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 ab16f4b9d68..1717e553f5c 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 @@ WithCTE
       +- Project [t1a#x, t1b#x, t1a#x, t1b#x]
          +- Join Inner, (t1b#x = t1b#x)
             :- SubqueryAlias cte1
-            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
             +- SubqueryAlias cte2
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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 9d82c707177..6d0a944bfcf 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 @@ WithCTE
       :  +- Project [t1b#x]
       :     +- Filter (cast(t1b#x as int) > 0)
       :        +- SubqueryAlias cte1
-      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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 @@ WithCTE
          :        :  :     :- Project [t1b#x]
          :        :  :     :  +- Filter (cast(t1b#x as int) > 0)
          :        :  :     :     +- SubqueryAlias cte1
-         :        :  :     :        +- CTERelationRef xxxx, true, [t1a#x, 
t1b#x]
+         :        :  :     :        +- CTERelationRef xxxx, true, [t1a#x, 
t1b#x], false
          :        :  :     +- Project [t1b#x]
          :        :  :        +- Filter (cast(t1b#x as int) > 5)
          :        :  :           +- SubqueryAlias cte1
-         :        :  :              +- CTERelationRef xxxx, true, [t1a#x, 
t1b#x]
+         :        :  :              +- CTERelationRef xxxx, true, [t1a#x, 
t1b#x], false
          :        :  +- Intersect false
          :        :     :- Project [t1b#x]
          :        :     :  +- SubqueryAlias cte1
-         :        :     :     +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+         :        :     :     +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 
false
          :        :     +- Project [t1b#x]
          :        :        +- SubqueryAlias cte1
-         :        :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+         :        :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 
false
          :        +- Project [t1b#x]
          :           +- SubqueryAlias cte1
-         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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 @@ WithCTE
       :        :  :  :- 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]
+      :        :  :  :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, 
t1c#x, t1d#xL, t1e#x], false
       :        :  :  :  :  +- SubqueryAlias cte2
       :        :  :  :  :     +- SubqueryAlias cte1
-      :        :  :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, 
t1c#x, t1d#xL, t1e#x]
+      :        :  :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, 
t1c#x, t1d#xL, t1e#x], false
       :        :  :  :  +- SubqueryAlias cte3
       :        :  :  :     +- SubqueryAlias cte1
-      :        :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, 
t1c#x, t1d#xL, t1e#x]
+      :        :  :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, 
t1c#x, t1d#xL, t1e#x], false
       :        :  :  +- SubqueryAlias cte4
       :        :  :     +- SubqueryAlias cte1
-      :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1e#x]
+      :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1e#x], false
       :        :  +- SubqueryAlias cte5
       :        :     +- SubqueryAlias cte1
-      :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1e#x]
+      :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1e#x], false
       :        +- SubqueryAlias cte6
       :           +- SubqueryAlias cte1
-      :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1e#x]
+      :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1e#x], 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 @@ WithCTE
             :- 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]
+            :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
             :  :  +- SubqueryAlias cte2
             :  :     +- SubqueryAlias cte1
-            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
             :  +- SubqueryAlias cte3
             :     +- SubqueryAlias cte1
-            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
             +- SubqueryAlias cte4
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
 
 
 -- !query
@@ -424,10 +424,10 @@ WithCTE
          +- Project [t1a#x, t1b#x]
             +- Join Inner, (t1h#x >= t1h#x)
                :- SubqueryAlias cte1
-               :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x]
+               :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false
                +- SubqueryAlias cte2
                   +- SubqueryAlias cte1
-                     +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x]
+                     +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false
 
 
 -- !query
@@ -485,16 +485,16 @@ WithCTE
             :- Join RightOuter, (t1b#x = t1b#x)
             :  :- Join Inner, (t1a#x = t1a#x)
             :  :  :- SubqueryAlias cte1
-            :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x]
+            :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false
             :  :  +- SubqueryAlias cte2
             :  :     +- SubqueryAlias cte1
-            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x]
+            :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], 
false
             :  +- SubqueryAlias cte3
             :     +- SubqueryAlias cte1
-            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x]
+            :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false
             +- SubqueryAlias cte4
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x]
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false
 
 
 -- !query
@@ -538,10 +538,10 @@ WithCTE
       +- Project [t1a#x, t1b#x]
          +- Join RightOuter, (t1a#x = t1a#x)
             :- SubqueryAlias cte1
-            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+            :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
             +- SubqueryAlias cte2
                +- SubqueryAlias cte1
-                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+                  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
 
 
 -- !query
@@ -599,15 +599,15 @@ WithCTE
          :        :           +- 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]
+         :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
          +- SubqueryAlias s
             +- Project [t1b#x]
                +- Join LeftOuter, (t1b#x = t1b#x)
                   :- SubqueryAlias cte1
-                  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+                  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
                   +- SubqueryAlias cte2
                      +- SubqueryAlias cte1
-                        +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+                        +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false
 
 
 -- !query
@@ -642,7 +642,7 @@ WithCTE
       :  +- Project [t1b#x]
       :     +- Filter (cast(t1b#x as int) < 0)
       :        +- SubqueryAlias cte1
-      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x]
+      :           +- CTERelationRef xxxx, true, [t1a#x, t1b#x], 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 @@ WithCTE
          :        :- 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]
+         :        :  :  :  +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1h#x], false
          :        :  :  +- SubqueryAlias cte2
          :        :  :     +- SubqueryAlias cte1
-         :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, 
t1c#x, t1d#xL, t1h#x]
+         :        :  :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, 
t1c#x, t1d#xL, t1h#x], false
          :        :  +- SubqueryAlias cte3
          :        :     +- SubqueryAlias cte1
-         :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1h#x]
+         :        :        +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1h#x], false
          :        +- SubqueryAlias cte4
          :           +- SubqueryAlias cte1
-         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1h#x]
+         :              +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, 
t1d#xL, t1h#x], 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 18c879466e3..5265b6163bd 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 @@ Project [c1#x, scalar-subquery#x [c1#x] AS 
scalarsubquery(c1)#x]
 :     :        +- OneRowRelation
 :     +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [a#x]
+:           +- CTERelationRef xxxx, true, [a#x], 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 @@ Project [c1#x, scalar-subquery#x [c1#x] AS 
scalarsubquery(c1)#xL]
 :     :                       +- LocalRelation [c1#x, c2#x]
 :     +- Aggregate [sum(c2#x) AS sum(c2)#xL]
 :        +- SubqueryAlias t
-:           +- CTERelationRef xxxx, true, [c1#x, c2#x]
+:           +- CTERelationRef xxxx, true, [c1#x, c2#x], 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, scalar-subquery#x [c1#x] AS 
scalarsubquery(c1)#xL]
 :     :     +- Project [c1#x, c2#x]
 :     :        +- Filter (outer(c1#x) = c1#x)
 :     :           +- SubqueryAlias t3
-:     :              +- CTERelationRef xxxx, true, [c1#x, c2#x]
+:     :              +- CTERelationRef xxxx, true, [c1#x, c2#x], false
 :     +- Aggregate [sum(c2#x) AS sum(c2)#xL]
 :        +- SubqueryAlias t4
-:           +- CTERelationRef xxxx, true, [c1#x, c2#x]
+:           +- CTERelationRef xxxx, true, [c1#x, c2#x], 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 @@ Project [c1#x, scalar-subquery#x [c1#x] AS 
scalarsubquery(c1)#xL]
 :                    +- Union false, false
 :                       :- Project [c1#x, c2#x]
 :                       :  +- SubqueryAlias t
-:                       :     +- CTERelationRef xxxx, true, [c1#x, c2#x]
+:                       :     +- CTERelationRef xxxx, true, [c1#x, c2#x], false
 :                       +- Project [c2#x, c1#x]
 :                          +- SubqueryAlias t
-:                             +- CTERelationRef xxxx, true, [c1#x, c2#x]
+:                             +- CTERelationRef xxxx, true, [c1#x, c2#x], 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 @@ WithCTE
       :           :  +- Aggregate [sum(c2#x) AS sum(c2)#xL]
       :           :     +- Filter (c1#x = outer(c1#x))
       :           :        +- SubqueryAlias t
-      :           :           +- CTERelationRef xxxx, true, [c1#x, c2#x]
+      :           :           +- CTERelationRef xxxx, true, [c1#x, c2#x], false
       :           +- SubqueryAlias v
-      :              +- CTERelationRef xxxx, true, [c1#x, c2#x]
+      :              +- CTERelationRef xxxx, true, [c1#x, c2#x], 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 @@ WithCTE
    :  +- Project [a#x]
    :     +- Filter (a#x = outer(c1#x))
    :        +- SubqueryAlias t
-   :           +- CTERelationRef xxxx, true, [a#x]
+   :           +- CTERelationRef xxxx, true, [a#x], 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 @@ WithCTE
    :  +- 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]
+   :           +- CTERelationRef xxxx, true, [a#x], 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 cda76f716a8..ceca433a1c9 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 @@ WithCTE
    +- Join Inner, (b#x = b#x)
       :- SubqueryAlias t1
       :  +- SubqueryAlias temp
-      :     +- CTERelationRef xxxx, true, [b#x]
+      :     +- CTERelationRef xxxx, true, [b#x], false
       +- SubqueryAlias t2
          +- SubqueryAlias temp
-            +- CTERelationRef xxxx, true, [b#x]
+            +- CTERelationRef xxxx, true, [b#x], 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 0fe7254d734..97410d3cdd3 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 @@ WithCTE
          +- Project [coalesce(key#x, key#x) AS key#x, key#x, key#x, key#x]
             +- Join FullOuter, (key#x = key#x)
                :- SubqueryAlias t1
-               :  +- CTERelationRef xxxx, true, [key#x]
+               :  +- CTERelationRef xxxx, true, [key#x], false
                +- SubqueryAlias t2
-                  +- CTERelationRef xxxx, true, [key#x]
+                  +- CTERelationRef xxxx, true, [key#x], false
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index e388de21405..b2978f25786 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, 
Row, SaveMode}
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, 
Shuffle, Uuid}
-import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, 
CTERelationRef, LocalRelation}
 import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Complete
 import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
 import org.apache.spark.sql.connector.read.InputPartition
@@ -1319,6 +1319,51 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
     }
   }
 
+  test("SPARK-46062: streaming query reading from CTE, which refers to temp 
view from " +
+    "streaming source") {
+    val inputStream = MemoryStream[Int]
+    inputStream.toDF().createOrReplaceTempView("tv")
+
+    val df = spark.sql(
+      """
+        |WITH w as (
+        |  SELECT * FROM tv
+        |)
+        |SELECT value from w
+        |""".stripMargin)
+
+    testStream(df)(
+      AddData(inputStream, 1, 2, 3),
+      CheckAnswer(1, 2, 3),
+      Execute { q =>
+        var isStreamingForCteDef: Option[Boolean] = None
+        var isStreamingForCteRef: Option[Boolean] = None
+
+        q.analyzedPlan.foreach {
+          case d: CTERelationDef =>
+            assert(d.resolved, "The definition node must be resolved after 
analysis.")
+            isStreamingForCteDef = Some(d.isStreaming)
+
+          case d: CTERelationRef =>
+            assert(d.resolved, "The reference node must be marked as resolved 
after analysis.")
+            isStreamingForCteRef = Some(d.isStreaming)
+
+          case _ =>
+        }
+
+        assert(isStreamingForCteDef.isDefined && 
isStreamingForCteRef.isDefined,
+          "Both definition and reference for CTE should be available in 
analyzed plan.")
+
+        assert(isStreamingForCteDef.get, "Expected isStreaming=true for CTE 
definition, but " +
+          "isStreaming is set to false.")
+
+        assert(isStreamingForCteDef === isStreamingForCteRef,
+          "isStreaming flag should be carried over from definition to 
reference, " +
+            s"definition: ${isStreamingForCteDef.get}, reference: 
${isStreamingForCteRef.get}.")
+      }
+    )
+  }
+
   private def checkExceptionMessage(df: DataFrame): Unit = {
     withTempDir { outputDir =>
       withTempDir { checkpointDir =>


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

Reply via email to