-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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-20175][SQL] Exists should not be evaluated in Join operator #17491
Changes from 3 commits
b012550
88016bf
329f067
24ae5ce
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 |
---|---|---|
|
@@ -498,3 +498,32 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { | |
} | ||
} | ||
} | ||
|
||
/** | ||
* This rule rewrites a EXISTS predicate sub-queries into an Aggregate with count. | ||
* So it doesn't be converted to a JOIN later. | ||
*/ | ||
object RewriteEmptyExists extends Rule[LogicalPlan] with PredicateHelper { | ||
private def containsAgg(plan: LogicalPlan): Boolean = { | ||
plan.collect { | ||
case a: Aggregate => a | ||
}.nonEmpty | ||
} | ||
|
||
def apply(plan: LogicalPlan): LogicalPlan = plan transform { | ||
case Filter(condition, child) => | ||
val (withSubquery, withoutSubquery) = | ||
splitConjunctivePredicates(condition).partition(SubqueryExpression.hasInOrExistsSubquery) | ||
val newWithSubquery = withSubquery.map(_.transform { | ||
case e @ Exists(sub, conditions, exprId) if conditions.isEmpty && !containsAgg(sub) => | ||
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. Currently take a conservative way, if there is already 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. The reason for this conversion is, I assume a count aggregation is cheaper than an existence join. 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. @viirya what we should do is to mark the outer join as an "early out" in the run-time join processing. An aggregation is not cheap as it needs to read the entire table to give the (first) answer. An "early out" logic is just like what we currently have in the LeftSemi join where only the first match of the join value is returned (and the subsequent matches are discarded). A LeftSemi join is a special case of an "early out" inner join where the columns of the right table are not permitted in the output of the join. IMO, a step forward is to implement the "early out" mechanism in all the run-time join operators, nested-loop, sorted-merge, and hash; and inner, left outer, right outer. Then LeftSemi and LeftAnti will just be special cases of one of those operators. 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. For a simple count aggregation, I think it is cheap because it should prune columns and no data from the table will be shuffled. 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. 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. A simple test will tell. If you try to do a count on a 1-billion row table versus 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. Yeah, that is a good idea. I think it definitely will help with an additional |
||
val countExpr = Alias(Count(Literal(1)).toAggregateExpression(), "count")() | ||
val expr = Alias(GreaterThan(countExpr.toAttribute, Literal(0)), e.toString)() | ||
ScalarSubquery( | ||
Project(Seq(expr), | ||
Aggregate(Nil, Seq(countExpr), LocalLimit(Literal(1), sub))), | ||
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. How useful is this optimization? It only works when I do agree with @nsyca that we should implement "early-out" mechanism, which is more general 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 think it is a special case. Then I will remove this optimization and minimize this pr's change. 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. We can address the early-out in other work. 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. Btw, I am not very sure this early-out can benefit the general usage, except for this kind of special case. |
||
children = Seq.empty, | ||
exprId = exprId) | ||
}) | ||
Filter((newWithSubquery ++ withoutSubquery).reduce(And), child) | ||
} | ||
} |
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.
@dilipbiswal Here is another case of a regression from #16954. Would you think we should just say the following?
case SubqueryExpression => false
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 think ScalarSubquery without correlated references can be pushed. Doesn't it?
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 am not sure. The name of this is
def canEvaluateWithinJoin
so I assume it asks whether an inputExpression
can be processed as part of a Join operator. Can aScalarSubquery
be processed inside a 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.
@nsyca @viirya I just verified and the exists test fails the same way in 2.1. So its not a regression.
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.
@nsyca Looking at this further, there is a SubqueryExec operator that can execute a ScalarSubquery and InSubquery (PlanSubqueries). As part of my change, i had removed the case for PredicateSubquery as we removed PredicateSubquery all together. I just quickly tried the following and got the query to work. I haven't verified the semantics but just tried something quickly. Basically if we were to keep the Exists expression as it is and push it down as a join condition and execute it as a InSubquery (possibly with a additional limit clause) there seems to be an infrastructure for it already. Or perhaps we may want to introduce a ExistSubquery exec operator that can work more efficiently.
What do you think Natt ?
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.
What this code does is around the idea of treating an uncorrelated subquery as a black box. The subquery is processed as a self-contained operation and a list of values is returned. After that, the code evaluates as if this is an IN list predicate like IN (). In your code above, is represented as a "true" literal. That means the returned values from the subquery must be in Boolean type too.
Putting a LIMIT does help to short-circuit the processing to the first row. I still think putting a LIMIT explicitly as an extra LogicalPlan operator may have some negative side effect in the way that it prevents other Optimizer rules to further optimize the query. I have not thought about a concrete example to back my belief though.
I feel this optimization could be done better in the run-time area, rather than trying to shoehorn it in the Optimizer phase. What we can do is 1) propagate the notion of "early out" deeper to the operator on the RHS of the outer join. If it's a scan, stop scanning on the first row. 2) one more step further: cache the result of the RHS without a rescan because the next row from the parent table will always get the same answer from rescanning the subquery.
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 remember
ScalarSubquery
without correlated reference will be evaluated as individual query plan and get its result back as an expression. So it should be no difference in run time compared with other expressions.A
Limit
looks good to me for now. I can't think a negative side effect prevents possible optimization for the subquery plan. Doesn't it just like a re-written query with a limit clause added?I think this is a corner usage case. To address this in run-time like the introduction of "early out" into physical join operators works, but it may involve a lot of code changes.
I quickly scan physical
SortMergeJoin
operator. If the streamed row matches the scanned group of rows, it will reuse the scanned group. Sounds it does what you said, if I don't miss something.I think current join operators are smart enough that they won't re-scan the subquery if the next row still matches the scaned group of rows.