-
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-18125][SQL] Fix a compilation error in codegen due to splitExpression #15693
Conversation
Test build #67807 has finished for PR 15693 at commit
|
// SPARK-18125: The children vars are local variables. If the result expression uses | ||
// splitExpression, those variables cannot be accessed so compilation fails. | ||
// To fix it, we use class variables to hold those local variables. | ||
val initClassChildVars = childrenVars.map { childVar => |
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.
Shouldn't we merge this with childrenVars
?
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.
We could. Not sure if merging them will reduce readability or not.
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.
Let me refactor it a bit.
Test build #67808 has finished for PR 15693 at commit
|
retest this please. |
Test build #67811 has finished for PR 15693 at commit
|
Test build #67850 has finished for PR 15693 at commit
|
@hvanhovell any more thoughts? |
Would it be possible to put a part of generated code before and after this PR? |
@kiszk the part of generated codes are posted as follows. Before this patch: Notice that line 239
After this patch:
|
Thanks, good to see generated code for ease of understanding. |
(childVar, classChildVar) | ||
}.unzip | ||
|
||
val initClassChildrenVars = classChildrenVars.zipWithIndex.map { case (classChildrenVar, i) => |
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.
Why not generate this code in the previous block?
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 want to avoid tangling all codes together. I made the change to generate the code in previous block. Please see if it is better or not.
Test build #68173 has finished for PR 15693 at commit
|
ping @hvanhovell Please let me know if you have other more comments. Thanks. |
Test build #68269 has finished for PR 15693 at commit
|
…ression ## What changes were proposed in this pull request? As reported in the jira, sometimes the generated java code in codegen will cause compilation error. Code snippet to test it: case class Route(src: String, dest: String, cost: Int) case class GroupedRoutes(src: String, dest: String, routes: Seq[Route]) val ds = sc.parallelize(Array( Route("a", "b", 1), Route("a", "b", 2), Route("a", "c", 2), Route("a", "d", 10), Route("b", "a", 1), Route("b", "a", 5), Route("b", "c", 6)) ).toDF.as[Route] val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r))) .groupByKey(r => (r.src, r.dest)) .reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) => GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes) }.map(_._2) The problem here is, in `ReferenceToExpressions` we evaluate the children vars to local variables. Then the result expression is evaluated to use those children variables. In the above case, the result expression code is too long and will be split by `CodegenContext.splitExpression`. So those local variables cannot be accessed and cause compilation error. ## How was this patch tested? Jenkins tests. Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. Author: Liang-Chi Hsieh <[email protected]> Closes #15693 from viirya/fix-codege-compilation-error. (cherry picked from commit a814eea) Signed-off-by: Herman van Hovell <[email protected]>
LGTM. Merging to master/2.1. Thanks! Could you open a backport for 2.0? |
@hvanhovell OK. I will open a backport later. |
## What changes were proposed in this pull request? ~In `TypedAggregateExpression.evaluateExpression`, we may create `ReferenceToExpressions` with `CreateStruct`, and `CreateStruct` may generate too many codes and split them into several methods. `ReferenceToExpressions` will replace `BoundReference` in `CreateStruct` with `LambdaVariable`, which can only be used as local variables and doesn't work if we split the generated code.~ It's already fixed by #15693 , this pr adds regression test ## How was this patch tested? new test in `DatasetAggregatorSuite` Author: Wenchen Fan <[email protected]> Closes #15807 from cloud-fan/typed-agg. (cherry picked from commit 6021c95) Signed-off-by: Wenchen Fan <[email protected]>
## What changes were proposed in this pull request? ~In `TypedAggregateExpression.evaluateExpression`, we may create `ReferenceToExpressions` with `CreateStruct`, and `CreateStruct` may generate too many codes and split them into several methods. `ReferenceToExpressions` will replace `BoundReference` in `CreateStruct` with `LambdaVariable`, which can only be used as local variables and doesn't work if we split the generated code.~ It's already fixed by #15693 , this pr adds regression test ## How was this patch tested? new test in `DatasetAggregatorSuite` Author: Wenchen Fan <[email protected]> Closes #15807 from cloud-fan/typed-agg.
…ression ## What changes were proposed in this pull request? As reported in the jira, sometimes the generated java code in codegen will cause compilation error. Code snippet to test it: case class Route(src: String, dest: String, cost: Int) case class GroupedRoutes(src: String, dest: String, routes: Seq[Route]) val ds = sc.parallelize(Array( Route("a", "b", 1), Route("a", "b", 2), Route("a", "c", 2), Route("a", "d", 10), Route("b", "a", 1), Route("b", "a", 5), Route("b", "c", 6)) ).toDF.as[Route] val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r))) .groupByKey(r => (r.src, r.dest)) .reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) => GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes) }.map(_._2) The problem here is, in `ReferenceToExpressions` we evaluate the children vars to local variables. Then the result expression is evaluated to use those children variables. In the above case, the result expression code is too long and will be split by `CodegenContext.splitExpression`. So those local variables cannot be accessed and cause compilation error. ## How was this patch tested? Jenkins tests. Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. Author: Liang-Chi Hsieh <[email protected]> Closes apache#15693 from viirya/fix-codege-compilation-error.
## What changes were proposed in this pull request? ~In `TypedAggregateExpression.evaluateExpression`, we may create `ReferenceToExpressions` with `CreateStruct`, and `CreateStruct` may generate too many codes and split them into several methods. `ReferenceToExpressions` will replace `BoundReference` in `CreateStruct` with `LambdaVariable`, which can only be used as local variables and doesn't work if we split the generated code.~ It's already fixed by apache#15693 , this pr adds regression test ## How was this patch tested? new test in `DatasetAggregatorSuite` Author: Wenchen Fan <[email protected]> Closes apache#15807 from cloud-fan/typed-agg.
What changes were proposed in this pull request?
As reported in the jira, sometimes the generated java code in codegen will cause compilation error.
Code snippet to test it:
The problem here is, in
ReferenceToExpressions
we evaluate the children vars to local variables. Then the result expression is evaluated to use those children variables. In the above case, the result expression code is too long and will be split byCodegenContext.splitExpression
. So those local variables cannot be accessed and cause compilation error.How was this patch tested?
Jenkins tests.
Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.