-
Notifications
You must be signed in to change notification settings - Fork 28k
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-16208][SQL] Add PropagateEmptyRelation
optimizer
#13906
Conversation
Test build #61242 has finished for PR 13906 at commit
|
case x if x.isInstanceOf[ObjectProducer] || x.isInstanceOf[ObjectConsumer] => x | ||
|
||
// Case 1: If groupingExpressions contains all aggregation expressions, the result is empty. | ||
case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && ae.forall(ge.contains(_)) => |
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.
this kind of blacklisting approach is too risky -- if we were to introduce a new logical node in the future, most likely we will forget to update this rule.
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.
Thank you for review, @rxin .
I see. I will update this PR into whitelist approach.
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.
Ur, at the first, I thought you meant line 1080 for case p: LogicalPlan
,
https://github.com/apache/spark/pull/13906/files#diff-a636a87d8843eeccca90140be91d4fafR1080 .
Did I understand your advice correctly?
Test build #61260 has finished for PR 13906 at commit
|
Hi, @rxin . |
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} | ||
import org.apache.spark.sql.catalyst.rules.RuleExecutor | ||
|
||
class CollapseEmptyPlanSuite extends PlanTest { |
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.
you should test something that shouldn't have been converted too
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.
Ur, any other scenario except the existing followings?
- test("one non-empty local relation")
- test("one non-empty and one empty local relations")
- test("aggregating expressions on empty plan")
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'll update to have more.
Anyway, thank you for review again, @rxin ! |
Test build #61363 has finished for PR 13906 at commit
|
// Case 4: The following plans having at least one empty relation return empty results. | ||
case p: LogicalPlan if p.children.exists(isEmptyLocalRelation) => | ||
p match { | ||
case Join(_, _, Inner, _) | _: Intersect => |
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.
can we move this out rather than doing a two level nesting
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.
Sure!
cc @cloud-fan to take a look too. |
* SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY a (Not optimized) | ||
* }}} | ||
*/ | ||
object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper { |
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.
can you actually move this into a separate file? the optimizer is becoming too large and I want to break it apart soon. No point adding new things in this file.
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.
Sure. What name is suitable for this optimizer?
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.
Just CollapseEmptyPlan.scala
is okay?
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.
yup
Yep. It's done! |
Test build #61409 has finished for PR 13906 at commit
|
Some more thoughts about it: Currently we only execute |
Thank you, @cloud-fan . |
Sounds interesting. You mean |
It sounds promising. Maybe, Spark 2.1? |
By the way, for complexity, it's 23 line optimizer without blank/comments. In fact, it's less than |
For 3, I respect your opinion. I just make another commit for 2. Now, it's 19 lines. |
Test build #61449 has finished for PR 13906 at commit
|
Test build #61462 has finished for PR 13906 at commit
|
|
||
case p: LogicalPlan if p.children.nonEmpty && p.children.forall(isEmptyLocalRelation) => | ||
p match { | ||
case _: Project | _: Generate | _: Filter | _: Sample | _: Join | |
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.
Actually Generate
can't be included here. Our Generate
also supports Hive style UDTF, which has a weird semantics: for a UDTF f
, after all rows being processed, f.close()
will be called, and more rows can be generated within f.close()
. This means a UDTF may generate one or more rows even if the underlying input is empty.
My feeling is that, this optimization rule is mostly useful for binary plan nodes like inner join and intersection, where we can avoid scanning output of the non-empty side. On the other hand, for unary plan nodes, firstly it doesn't bring much performance benefits, especially when whole stage codegen is enabled; secondly there are non-obvious and tricky corner cases, like That said, although this patch is not a big one, it does introduce non-trivial complexities. For example, I didn't immediately realize that why So my suggestion is to only implement this rule for inner join and intersection, which are much simpler to handle. what do you think? |
*/ | ||
object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper { | ||
private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = | ||
plan.isInstanceOf[LocalRelation] && plan.asInstanceOf[LocalRelation].data.isEmpty |
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.
plan match {
case p: LocalRelation => p.data.isEmpty
case _ => false
}
@liancheng , I think we still need to keep some simple rules for unary node, which also helps the binary cases, as the empty relation is propagated up. |
@cloud-fan Yea, that's a good point. |
Thank you so much, @liancheng and @cloud-fan ! I update the code and PR description according to the comments.
|
Test build #61581 has finished for PR 13906 at commit
|
Oh, please hold on merging this PR. Scala 2.10 reports some errors. I'll inform you again. |
It's the exact same The solution is the same. I split the |
Test build #61598 has finished for PR 13906 at commit
|
Hi, @rxin , @cloud-fan , @liancheng . It's ready for merging again. |
case p: Union if p.children.forall(isEmptyLocalRelation) => | ||
empty(p) | ||
|
||
case p @ Join(_, _, joinType, _) if p.children.exists(isEmptyLocalRelation) => joinType match { |
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.
sorry to be so nit-pick, but it seems better to separate the join case, e.g.
case p @ Join(_, _, Inner, _) if p.children.exists(isEmptyLocalRelation) => empty(p)
case p @ Join(_, _, LeftOuter | LeftSemi | LeftAnti, _) if isEmptyLocalRelation(p.left) => empty(p)
...
Merged to master. @cloud-fan Sorry that I didn't notice your comment while merging it. We may address it in follow-up ones. |
Thank you for merging, @liancheng , @cloud-fan , and @rxin . |
I found a bug in the part of this rule which deals with |
…te without grouping ## What changes were proposed in this pull request? The query ``` SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1 ``` should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows. This is caused by SPARK-16208 / apache#13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead: An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows. If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation. The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be). This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation. ## How was this patch tested? - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file. - Updated unit tests in `PropagateEmptyRelationSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes apache#17929 from JoshRosen/fix-PropagateEmptyRelation.
…te without grouping ## What changes were proposed in this pull request? The query ``` SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1 ``` should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows. This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead: An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows. If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation. The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be). This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation. ## How was this patch tested? - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file. - Updated unit tests in `PropagateEmptyRelationSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes #17929 from JoshRosen/fix-PropagateEmptyRelation. (cherry picked from commit a90c5cd) Signed-off-by: Wenchen Fan <wenchen@databricks.com>
…te without grouping The query ``` SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1 ``` should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows. This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead: An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows. If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation. The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be). This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation. - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file. - Updated unit tests in `PropagateEmptyRelationSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes #17929 from JoshRosen/fix-PropagateEmptyRelation. (cherry picked from commit a90c5cd) Signed-off-by: Wenchen Fan <wenchen@databricks.com>
…te without grouping ## What changes were proposed in this pull request? The query ``` SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1 ``` should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows. This is caused by SPARK-16208 / apache#13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead: An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows. If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation. The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be). This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation. ## How was this patch tested? - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file. - Updated unit tests in `PropagateEmptyRelationSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes apache#17929 from JoshRosen/fix-PropagateEmptyRelation.
…te without grouping The query ``` SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1 ``` should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows. This is caused by SPARK-16208 / apache#13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead: An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows. If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation. The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be). This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation. - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file. - Updated unit tests in `PropagateEmptyRelationSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes apache#17929 from JoshRosen/fix-PropagateEmptyRelation. (cherry picked from commit a90c5cd) Signed-off-by: Wenchen Fan <wenchen@databricks.com>
What changes were proposed in this pull request?
This PR adds a new logical optimizer,
PropagateEmptyRelation
, to collapse a logical plans consisting of only empty LocalRelations.Optimizer Targets
Sample Query
Before
After
How was this patch tested?
Pass the Jenkins tests (including a new testsuite).