-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-46741][SQL] Cache Table with CTE won't work #53333
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
c151b60
b9711ab
38c68d8
d587e6d
8b2a25d
bb15d32
fa17613
a6e2657
c9017ef
4aa6aee
99bf379
054231f
92a213d
123a986
611b1b4
5448b73
99ee735
6a92b32
5770ff3
7243d1f
40a4332
95bcf5d
2b170fd
1adab54
f99feb1
de0a94d
d10fd2b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,54 @@ | ||
| /* | ||
| * 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.normalizer | ||
|
|
||
| import org.apache.spark.sql.catalyst.plans.logical.{CacheTableAsSelect, CTERelationRef, LogicalPlan, UnionLoop, UnionLoopRef, WithCTE} | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
|
|
||
| object NormalizeCTEIds extends Rule[LogicalPlan]{ | ||
| override def apply(plan: LogicalPlan): LogicalPlan = { | ||
| val curId = new java.util.concurrent.atomic.AtomicLong() | ||
| plan transformDown { | ||
|
|
||
| case ctas @ CacheTableAsSelect(_, plan, _, _, _, _, _) => | ||
| ctas.copy(plan = apply(plan)) | ||
|
|
||
| case withCTE @ WithCTE(plan, cteDefs) => | ||
| val defIdToNewId = withCTE.cteDefs.map(_.id).map((_, curId.getAndIncrement())).toMap | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why we use an
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm, as you normalize
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I can't come up with an exact example where processing |
||
| val normalizedPlan = canonicalizeCTE(plan, defIdToNewId) | ||
| val newCteDefs = cteDefs.map { cteDef => | ||
| val normalizedCteDef = canonicalizeCTE(cteDef.child, defIdToNewId) | ||
| cteDef.copy(child = normalizedCteDef, id = defIdToNewId(cteDef.id)) | ||
| } | ||
| withCTE.copy(plan = normalizedPlan, cteDefs = newCteDefs) | ||
| } | ||
| } | ||
|
|
||
| def canonicalizeCTE(plan: LogicalPlan, defIdToNewId: Map[Long, Long]): LogicalPlan = { | ||
| plan.transformDownWithSubqueries { | ||
| // For nested WithCTE, if defIndex didn't contain the cteId, | ||
| // means it's not current WithCTE's ref. | ||
| case ref: CTERelationRef if defIdToNewId.contains(ref.cteId) => | ||
AngersZhuuuu marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| ref.copy(cteId = defIdToNewId(ref.cteId)) | ||
| case unionLoop: UnionLoop if defIdToNewId.contains(unionLoop.id) => | ||
| unionLoop.copy(id = defIdToNewId(unionLoop.id)) | ||
| case unionLoopRef: UnionLoopRef if defIdToNewId.contains(unionLoopRef.loopId) => | ||
| unionLoopRef.copy(loopId = defIdToNewId(unionLoopRef.loopId)) | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
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
WithCTEnodes in subquery expressions?There was a problem hiding this comment.
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
transformDownWithSubquerieshere as well.There was a problem hiding this comment.
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?