-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-43838][SQL] Fix subquery on single table with having clause can't be optimized #41347
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
0c12ef9
0b3fce7
56f8fd6
7b2521c
76b84b5
d265a28
9b37afd
9822c0a
f324892
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 |
|---|---|---|
|
|
@@ -271,7 +271,10 @@ case class ScalarSubquery( | |
| mayHaveCountBug: Option[Boolean] = None) | ||
| extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Unevaluable { | ||
| override def dataType: DataType = { | ||
| assert(plan.schema.fields.nonEmpty, "Scalar subquery should have only one column") | ||
| if (!plan.schema.fields.nonEmpty) { | ||
|
Member
Author
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. make sure the
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. Can we really reach this code branch?
Member
Author
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. Yes. Usually this error will be thrown by Change before: |
||
| throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length, | ||
| origin) | ||
| } | ||
| plan.schema.fields.head.dataType | ||
| } | ||
| override def nullable: Boolean = true | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -437,25 +437,6 @@ class LeftSemiAntiJoinPushDownSuite extends PlanTest { | |
| } | ||
| } | ||
|
|
||
| Seq(LeftSemi, LeftAnti).foreach { case jt => | ||
|
Member
Author
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. This test unnecessary. Because we can deduplicate those attributes in anti-join / semi-join is a self-join. Please refer #39131 |
||
| test(s"Aggregate: $jt join no pushdown - join condition refers left leg and right leg child") { | ||
| val aggregation = testRelation | ||
| .select($"b".as("id"), $"c") | ||
| .groupBy($"id")($"id", sum($"c").as("sum")) | ||
|
|
||
| // reference "b" exists in left leg, and the children of the right leg of the join | ||
| val originalQuery = aggregation.select(($"id" + 1).as("id_plus_1"), $"sum") | ||
| .join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1")) | ||
| val optimized = Optimize.execute(originalQuery.analyze) | ||
| val correctAnswer = testRelation | ||
| .select($"b".as("id"), $"c") | ||
| .groupBy($"id")(($"id" + 1).as("id_plus_1"), sum($"c").as("sum")) | ||
| .join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1")) | ||
| .analyze | ||
| comparePlans(optimized, correctAnswer) | ||
| } | ||
| } | ||
|
|
||
| Seq(LeftSemi, LeftAnti).foreach { case outerJT => | ||
| Seq(Inner, LeftOuter, RightOuter, Cross).foreach { case innerJT => | ||
| test(s"$outerJT no pushdown - join condition refers none of the leg - join type $innerJT") { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.