Skip to content

Conversation

@AngersZhuuuu
Copy link
Contributor

@AngersZhuuuu AngersZhuuuu commented Dec 5, 2025

What changes were proposed in this pull request?

Reopen #44767
Cache Table with CTE won't work, there are two reasons

  1. In the current code CTE in CacheTableAsSelect will be inlined
  2. CTERelation Ref and Def didn't handle the CTEId doCanonicalize issue
    Cause the current case can't be matched.

Why are the changes needed?

Fix Bug

Does this PR introduce any user-facing change?

Yea, Cache table with CTE can work after this pr

For added cache.sql final query
EXPLAIN EXTENDED SELECT * FROM cache_nested_cte_table;

Before this pr, the plan as below, cache won't work.
截屏2025-12-05 11 22 05

After this pr
截屏2025-12-05 11 32 38

How was this patch tested?

Added UT

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

No

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

@cloud-fan Could you take a look again? in #44767 you approved but not merge this pr.
This bug really impact product job's performance

@AngersZhuuuu
Copy link
Contributor Author

ping @cloud-fan

ctas.copy(plan = apply(plan))

case withCTE @ WithCTE(plan, cteDefs) =>
val defIdToNewId = withCTE.cteDefs.map(_.id).map((_, curId.getAndIncrement())).toMap
Copy link
Contributor

@peter-toth peter-toth Dec 12, 2025

Choose a reason for hiding this comment

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

Why we use an AtomicLong.getAndIncrement() here? Could a simple var work here?
Anyways, this is just a nit.

Copy link
Contributor

@peter-toth peter-toth Dec 12, 2025

Choose a reason for hiding this comment

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

Hmm, as you normalize WithCTE nodes one by one, but the nodes can be nested, once you normalized one of them, can the normalized ids conflict with the not yet normalized ids of an other node?

Copy link
Contributor

Choose a reason for hiding this comment

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

I can't come up with an exact example where processing WithCTE nodes one by one is an issue, but I wonder if traversing the whole plan and maintaining a global replacement map would be safer and more efficient solution.

@cloud-fan
Copy link
Contributor

thanks, merging to master/4.1!

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

Choose a reason for hiding this comment

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

Does this rule normalize ids of WithCTE nodes in subquery expressions?

Copy link
Contributor

Choose a reason for hiding this comment

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

ah missed it, we should use transformDownWithSubqueries here as well.

Copy link
Contributor

Choose a reason for hiding this comment

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

@AngersZhuuuu can you create a followup?

@cloud-fan cloud-fan closed this in 8f69679 Dec 18, 2025
cloud-fan pushed a commit that referenced this pull request Dec 18, 2025
### What changes were proposed in this pull request?
Reopen #44767
Cache Table with CTE won't work, there are two reasons
  1. In the current code CTE in CacheTableAsSelect will be inlined
  2. CTERelation Ref and Def didn't handle the CTEId doCanonicalize issue
Cause the current case can't be matched.

### Why are the changes needed?
Fix Bug

### Does this PR introduce _any_ user-facing change?
Yea, Cache table with CTE can work after this pr

For added `cache.sql` final query
`EXPLAIN EXTENDED SELECT * FROM cache_nested_cte_table;`

Before this pr, the plan as below, cache won't work.
<img width="1067" height="584" alt="截屏2025-12-05 11 22 05" src="https://github.com/user-attachments/assets/045df794-38e2-47d9-848e-cfc3c7525671" />

After this pr
<img width="1279" height="824" alt="截屏2025-12-05 11 32 38" src="https://github.com/user-attachments/assets/86f5ab33-67c6-44d0-b5d8-4bec51a2d5b7" />

### How was this patch tested?
Added UT

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

Closes #53333 from AngersZhuuuu/SPARK-46741.

Authored-by: Angerszhuuuu <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
(cherry picked from commit 8f69679)
Signed-off-by: Wenchen Fan <[email protected]>
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]>
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.

3 participants