-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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
[SPARK-48307][SQL] InlineCTE should keep not-inlined relations in the original WithCTE node #46617
Conversation
outerCTEId.foreach { cteId => | ||
val (_, _, outerRefMap) = cteMap(cteId) | ||
outerRefMap(ref.cteId) += 1 | ||
cteMap(cteId).recordOutgoingReference(ref.cteId) |
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.
I personally like to not hide the +1
implementation details here as I had troubles to understand it.
Or the function name can be IncreaseOutgoingReferenceByOne
?
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.
LGTM
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTESuite.scala
Outdated
Show resolved
Hide resolved
Repartition? Do you mean Relation? |
…mizer/InlineCTESuite.scala Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
It's Reparition, because we rely on shuffle reuse to reuse CTE relations. |
thanks for the review, merging to master! |
…CTE relations ### What changes were proposed in this pull request? This is a followup of #46617 . Subquery expression has a bunch of correlation checks which need to match certain plan shapes. We broke this by leaving `WithCTE` in the plan for un-referenced CTE relations. This PR fixes the issue by skipping CTE plan nodes in correlated subquery expression checks. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? no bug is not released yet. ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #46869 from cloud-fan/check. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
…d not fail ### What changes were proposed in this pull request? This is a follow-up of #46617 to fix a bug. When we re-construct the `WithCTE` node, we should use the new CTE definitions that have been applied `inlineCTE`. ### Why are the changes needed? bug fix, otherwise we may hit errors such as ``` java.util.NoSuchElementException: key not found: 0 at scala.collection.MapOps.default(Map.scala:289) at scala.collection.MapOps.default$(Map.scala:288) at scala.collection.AbstractMap.default(Map.scala:420) at scala.collection.mutable.HashMap.apply(HashMap.scala:440) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.gatherPredicatesAndAttributes(PushdownPredicatesAndPruneColumnsForCTEDef.scala:74) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.$anonfun$gatherPredicatesAndAttributes$1(PushdownPredicatesAndPruneColumnsForCTEDef.scala:68) at scala.collection.immutable.Vector.foreach(Vector.scala:2124) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.gatherPredicatesAndAttributes(PushdownPredicatesAndPruneColumnsForCTEDef.scala:67) ``` ### Does this PR introduce _any_ user-facing change? no, the bug is not released yet ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes #47141 from cloud-fan/fix. Lead-authored-by: Wenchen Fan <wenchen@databricks.com> Co-authored-by: Wenchen Fan <cloud0fan@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
…d not fail ### What changes were proposed in this pull request? This is a follow-up of apache#46617 to fix a bug. When we re-construct the `WithCTE` node, we should use the new CTE definitions that have been applied `inlineCTE`. ### Why are the changes needed? bug fix, otherwise we may hit errors such as ``` java.util.NoSuchElementException: key not found: 0 at scala.collection.MapOps.default(Map.scala:289) at scala.collection.MapOps.default$(Map.scala:288) at scala.collection.AbstractMap.default(Map.scala:420) at scala.collection.mutable.HashMap.apply(HashMap.scala:440) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.gatherPredicatesAndAttributes(PushdownPredicatesAndPruneColumnsForCTEDef.scala:74) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.$anonfun$gatherPredicatesAndAttributes$1(PushdownPredicatesAndPruneColumnsForCTEDef.scala:68) at scala.collection.immutable.Vector.foreach(Vector.scala:2124) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.gatherPredicatesAndAttributes(PushdownPredicatesAndPruneColumnsForCTEDef.scala:67) ``` ### Does this PR introduce _any_ user-facing change? no, the bug is not released yet ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes apache#47141 from cloud-fan/fix. Lead-authored-by: Wenchen Fan <wenchen@databricks.com> Co-authored-by: Wenchen Fan <cloud0fan@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
late LGTM |
…d not fail ### What changes were proposed in this pull request? This is a follow-up of apache#46617 to fix a bug. When we re-construct the `WithCTE` node, we should use the new CTE definitions that have been applied `inlineCTE`. ### Why are the changes needed? bug fix, otherwise we may hit errors such as ``` java.util.NoSuchElementException: key not found: 0 at scala.collection.MapOps.default(Map.scala:289) at scala.collection.MapOps.default$(Map.scala:288) at scala.collection.AbstractMap.default(Map.scala:420) at scala.collection.mutable.HashMap.apply(HashMap.scala:440) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.gatherPredicatesAndAttributes(PushdownPredicatesAndPruneColumnsForCTEDef.scala:74) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.$anonfun$gatherPredicatesAndAttributes$1(PushdownPredicatesAndPruneColumnsForCTEDef.scala:68) at scala.collection.immutable.Vector.foreach(Vector.scala:2124) at org.apache.spark.sql.catalyst.optimizer.PushdownPredicatesAndPruneColumnsForCTEDef$.gatherPredicatesAndAttributes(PushdownPredicatesAndPruneColumnsForCTEDef.scala:67) ``` ### Does this PR introduce _any_ user-facing change? no, the bug is not released yet ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes apache#47141 from cloud-fan/fix. Lead-authored-by: Wenchen Fan <wenchen@databricks.com> Co-authored-by: Wenchen Fan <cloud0fan@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
What changes were proposed in this pull request?
I noticed an outdated comment in the rule
InlineCTE
This is not true anymore after #42036 . It's not a big deal as we replace not-inlined CTE relations with
Repartition
during optimization, so it doesn't matter where we put theWithCTE
node with not-inlined CTE relations, as it will disappear eventually. But it's still better to keep it at its original place, as third-party rules may be sensitive about the plan shape.Why are the changes needed?
to keep the plan shape as much as can after inlining CTE relations.
Does this PR introduce any user-facing change?
no
How was this patch tested?
new test
Was this patch authored or co-authored using generative AI tooling?
no