Skip to content

Conversation

@AngersZhuuuu
Copy link
Contributor

What changes were proposed in this pull request?

Follow comment #53333 (comment)

Why are the changes needed?

Support all case

Does this PR introduce any user-facing change?

No

How was this patch tested?

UT

Was this patch authored or co-authored using generative AI tooling?

No

@github-actions github-actions bot added the SQL label Dec 18, 2025
@AngersZhuuuu
Copy link
Contributor Author

ping @cloud-fan , could you take a look? also add a corresponding unit suite case, verified that before change to transformDownWithSubqueries the test case will fail.

@cloud-fan
Copy link
Contributor

@AngersZhuuuu can you re-trigger failed CI jobs?

@AngersZhuuuu
Copy link
Contributor Author

@AngersZhuuuu can you re-trigger failed CI jobs?

DOne

object NormalizeCTEIds extends Rule[LogicalPlan]{
object NormalizeCTEIds extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = {
val curId = new java.util.concurrent.atomic.AtomicLong()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldn't we have unique CTE ids per query? then we need a new AtomicLong instance per apply invocation.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know this, directly change to transformDownWithSubqueries will cause UT SPARK-51109 failed.
For query

  test("SPARK-51109: CTE in subquery expression as grouping column") {
    withTable("t") {
      Seq(1 -> 1).toDF("c1", "c2").write.saveAsTable("t")
      withView("v") {
        sql(
          """
            |CREATE VIEW v AS
            |WITH r AS (SELECT c1 + c2 AS c FROM t)
            |SELECT * FROM r
            |""".stripMargin)
        checkAnswer(
          sql("SELECT (SELECT max(c) FROM v WHERE c > id) FROM range(1) GROUP BY 1"),
          Row(2)
        )
      }
    }
  }

Plan will be normalized from

Aggregate [scalar-subquery#15 [id#16L]], [scalar-subquery#15 [id#16L] AS scalarsubquery(id)#21]
:  :- Aggregate [max(c#18) AS max(c)#20]
:  :  +- Filter (cast(c#18 as bigint) > outer(id#16L))
:  :     +- SubqueryAlias spark_catalog.default.v
:  :        +- View (`spark_catalog`.`default`.`v`, [c#18])
:  :           +- Project [cast(c#17 as int) AS c#18]
:  :              +- WithCTE
:  :                 :- CTERelationDef 1, false
:  :                 :  +- SubqueryAlias r
:  :                 :     +- Project [(c1#12 + c2#13) AS c#17]
:  :                 :        +- SubqueryAlias spark_catalog.default.t
:  :                 :           +- Relation spark_catalog.default.t[c1#12,c2#13] parquet
:  :                 +- Project [c#17]
:  :                    +- SubqueryAlias r
:  :                       +- CTERelationRef 1, true, [c#17], false, false
:  +- Aggregate [max(c#26) AS max(c)#27]
:     +- Filter (cast(c#26 as bigint) > outer(id#16L))
:        +- SubqueryAlias spark_catalog.default.v
:           +- View (`spark_catalog`.`default`.`v`, [c#26])
:              +- Project [cast(c#25 as int) AS c#26]
:                 +- WithCTE
:                    :- CTERelationDef 1, false
:                    :  +- SubqueryAlias r
:                    :     +- Project [(c1#22 + c2#23) AS c#24]
:                    :        +- SubqueryAlias spark_catalog.default.t
:                    :           +- Relation spark_catalog.default.t[c1#22,c2#23] parquet
:                    +- Project [c#25]
:                       +- SubqueryAlias r
:                          +- CTERelationRef 1, true, [c#25], false, false
+- Range (0, 1, step=1)

to

Aggregate [scalar-subquery#15 [id#16L]], [scalar-subquery#15 [id#16L] AS scalarsubquery(id)#21]
:  :- Aggregate [max(c#18) AS max(c)#20]
:  :  +- Filter (cast(c#18 as bigint) > outer(id#16L))
:  :     +- SubqueryAlias spark_catalog.default.v
:  :        +- View (`spark_catalog`.`default`.`v`, [c#18])
:  :           +- Project [cast(c#17 as int) AS c#18]
:  :              +- WithCTE
:  :                 :- CTERelationDef 0, false
:  :                 :  +- SubqueryAlias r
:  :                 :     +- Project [(c1#12 + c2#13) AS c#17]
:  :                 :        +- SubqueryAlias spark_catalog.default.t
:  :                 :           +- Relation spark_catalog.default.t[c1#12,c2#13] parquet
:  :                 +- Project [c#17]
:  :                    +- SubqueryAlias r
:  :                       +- CTERelationRef 0, true, [c#17], false, false
:  +- Aggregate [max(c#26) AS max(c)#27]
:     +- Filter (cast(c#26 as bigint) > outer(id#16L))
:        +- SubqueryAlias spark_catalog.default.v
:           +- View (`spark_catalog`.`default`.`v`, [c#26])
:              +- Project [cast(c#25 as int) AS c#26]
:                 +- WithCTE
:                    :- CTERelationDef 1, false
:                    :  +- SubqueryAlias r
:                    :     +- Project [(c1#22 + c2#23) AS c#24]
:                    :        +- SubqueryAlias spark_catalog.default.t
:                    :           +- Relation spark_catalog.default.t[c1#22,c2#23] parquet
:                    +- Project [c#25]
:                       +- SubqueryAlias r
:                          +- CTERelationRef 1, true, [c#25], false, false
+- Range (0, 1, step=1)

in same plan the normalized cte id changed causing throw

[info]  is not a valid aggregate expression: [SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION] The correlated scalar subquery '"scalarsubquery(id)"' is neither present in GROUP BY, nor in an aggregate function.
[info] Add it to GROUP BY using ordinal position or wrap it in `first()` (or `first_value`) if you don't care which value you get. SQLSTATE: 0A000; line 1 pos 7
[info] Previous schema:scalarsubquery(id)#21

I am still trying how to fix such problem.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This means we should handle the case when CTE def IDs can be duplicated. In such cases, we should not generate new IDs blindly.

val defIdToNewId = withCTE.cteDefs.map(_.id).map((_, curId.getAndIncrement())).toMap

We need to fix this line. The id map should be per apply invocation, not per WithCTE.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Or use a global map in one traversal: #53333 (comment)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ping @cloud-fan How about current?

plan transformDown {
val defIdToNewId = new HashMap[Long, Long]()

plan transformDownWithSubqueries {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here I tried foreachWithSubqueries..but meet strange exception== @cloud-fan

Copy link
Contributor

@peter-toth peter-toth left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Only a minor nit.

@cloud-fan
Copy link
Contributor

thanks, merging to master/4.1!

@cloud-fan cloud-fan closed this in d65ee81 Dec 23, 2025
cloud-fan added a commit that referenced this pull request Dec 23, 2025
…expression subquery

### What changes were proposed in this pull request?
Follow comment #53333 (comment)

### Why are the changes needed?
Support all case

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
UT

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #53526 from AngersZhuuuu/SPARK-46741-FOLLOWUP.

Lead-authored-by: Angerszhuuuu <[email protected]>
Co-authored-by: Wenchen Fan <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
(cherry picked from commit d65ee81)
Signed-off-by: Wenchen Fan <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants