a814eeac6b
## 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 <viirya@gmail.com> Closes #15693 from viirya/fix-codege-compilation-error. |
||
---|---|---|
.. | ||
main | ||
test |