Skip to content
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

Closed
wants to merge 5 commits into from

Conversation

viirya
Copy link
Member

@viirya viirya commented Oct 31, 2016

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.

@SparkQA
Copy link

SparkQA commented Oct 31, 2016

Test build #67807 has finished for PR 15693 at commit cb876e4.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

// 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 =>
Copy link
Contributor

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?

Copy link
Member Author

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.

Copy link
Member Author

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.

@SparkQA
Copy link

SparkQA commented Oct 31, 2016

Test build #67808 has finished for PR 15693 at commit 0b660e0.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member Author

viirya commented Oct 31, 2016

retest this please.

@SparkQA
Copy link

SparkQA commented Oct 31, 2016

Test build #67811 has finished for PR 15693 at commit 0b660e0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 1, 2016

Test build #67850 has finished for PR 15693 at commit 5006bcb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member Author

viirya commented Nov 2, 2016

@hvanhovell any more thoughts?

@kiszk
Copy link
Member

kiszk commented Nov 2, 2016

Would it be possible to put a part of generated code before and after this PR?

@viirya
Copy link
Member Author

viirya commented Nov 4, 2016

@kiszk the part of generated codes are posted as follows.

Before this patch:

Notice that line 239 funcResult3 = value4._2();, value4 cannot be accessed, since it is locally defined in the method apply.

/* 288 */   public java.lang.Object apply(java.lang.Object _i) {
/* 289 */     InternalRow i = (InternalRow) _i;
/* 290 */
/* 291 */
/* 292 */
/* 293 */     Object obj = ((Expression) references[0]).eval(null);
/* 294 */     scala.Tuple2 value1 = (scala.Tuple2) obj;
/* 295 */
/* 296 */     boolean isNull2 = false;
/* 297 */     boolean value2 = false;
/* 298 */     if (!isNull2) {
/* 299 */
/* 300 */       Object funcResult = null;
/* 301 */       funcResult = value1._1();
/* 302 */       if (funcResult == null) {
/* 303 */         isNull2 = true;
/* 304 */       } else {
/* 305 */         value2 = (Boolean) funcResult;
/* 306 */       }
/* 307 */
/* 308 */     }
/* 309 */     this.isNull_0 = isNull2;
/* 310 */     this.value_0 = value2;
/* 311 */
/* 312 */
/* 313 */     Object obj1 = ((Expression) references[1]).eval(null);
/* 314 */     scala.Tuple2 value4 = (scala.Tuple2) obj1;
/* 315 */
/* 316 */     boolean isNull8 = false;
/* 317 */     org.apache.spark.sql.GroupedRoutes value8 = null;
/* 318 */     if (!isNull8) {
/* 319 */
/* 320 */       Object funcResult1 = null;
/* 321 */       funcResult1 = value4._2();
/* 322 */       if (funcResult1 == null) {
/* 323 */         isNull8 = true;
/* 324 */       } else {
/* 325 */         value8 = (org.apache.spark.sql.GroupedRoutes) funcResult1;
/* 326 */       }
/* 327 */
/* 328 */     }
...
/* 232 */   private void apply1_0(InternalRow i) {
/* 233 */
/* 234 */     boolean isNull17 = false;
/* 235 */     org.apache.spark.sql.GroupedRoutes value17 = null;
/* 236 */     if (!isNull17) {
/* 237 */
/* 238 */       Object funcResult3 = null;
/* 239 */       funcResult3 = value4._2();
/* 240 */       if (funcResult3 == null) {
/* 241 */         isNull17 = true;
/* 242 */       } else {
/* 243 */         value17 = (org.apache.spark.sql.GroupedRoutes) funcResult3;
/* 244 */       }
/* 245 */
/* 246 */     }
/* 247 */     isNull17 = value17 == null;
/* 248 */
/* 249 */     if (isNull17) {
/* 250 */       throw new RuntimeException(((java.lang.String) references[3]));
/* 251 */     }
/* 252 */

After this patch:

/* 292 */   public java.lang.Object apply(java.lang.Object _i) {
/* 293 */     InternalRow i = (InternalRow) _i;
/* 294 */
/* 295 */
/* 296 */
/* 297 */     Object obj = ((Expression) references[0]).eval(null);
/* 298 */     scala.Tuple2 value1 = (scala.Tuple2) obj;
/* 299 */     classChildVar = value1;
/* 300 */     classChildVarIsNull = false;
/* 301 */     boolean isNull2 = classChildVarIsNull;
/* 302 */     boolean value2 = false;
/* 303 */     if (!isNull2) {
/* 304 */
/* 305 */       Object funcResult = null;
/* 306 */       funcResult = classChildVar._1();
/* 307 */       if (funcResult == null) {
/* 308 */         isNull2 = true;
/* 309 */       } else {
/* 310 */         value2 = (Boolean) funcResult;
/* 311 */       }
/* 312 */
/* 313 */     }
/* 314 */     this.isNull_0 = isNull2;
/* 315 */     this.value_0 = value2;
/* 316 */
/* 317 */
/* 318 */     Object obj1 = ((Expression) references[1]).eval(null);
/* 319 */     scala.Tuple2 value4 = (scala.Tuple2) obj1;
/* 320 */     classChildVar1 = value4;
/* 321 */     classChildVarIsNull1 = false;
/* 322 */     boolean isNull8 = classChildVarIsNull1;
/* 323 */     org.apache.spark.sql.GroupedRoutes value8 = null;
/* 324 */     if (!isNull8) {
/* 325 */
/* 326 */       Object funcResult1 = null;
/* 327 */       funcResult1 = classChildVar1._2();
/* 328 */       if (funcResult1 == null) {
/* 329 */         isNull8 = true;
/* 330 */       } else {
/* 331 */         value8 = (org.apache.spark.sql.GroupedRoutes) funcResult1;
/* 332 */       }
/* 333 */
/* 334 */     }
/* 335 */     isNull8 = value8 == null;
...
/* 236 */   private void apply1_0(InternalRow i) {
/* 237 */
/* 238 */     boolean isNull17 = classChildVarIsNull1;
/* 239 */     org.apache.spark.sql.GroupedRoutes value17 = null;
/* 240 */     if (!isNull17) {
/* 241 */
/* 242 */       Object funcResult3 = null;
/* 243 */       funcResult3 = classChildVar1._2();
/* 244 */       if (funcResult3 == null) {
/* 245 */         isNull17 = true;
/* 246 */       } else {
/* 247 */         value17 = (org.apache.spark.sql.GroupedRoutes) funcResult3;
/* 248 */       }
/* 249 */
/* 250 */     }
/* 251 */     isNull17 = value17 == null;
/* 252 */
/* 253 */     if (isNull17) {
/* 254 */       throw new RuntimeException(((java.lang.String) references[3]));
/* 255 */     }
/* 256 */
/* 257 */     boolean isNull15 = false;
/* 258 */     java.lang.String value15 = null;
/* 259 */     if (!isNull15) {
/* 260 */
/* 261 */       Object funcResult4 = null;
/* 262 */       funcResult4 = value17.src();
/* 263 */       if (funcResult4 == null) {
/* 264 */         isNull15 = true;
/* 265 */       } else {
/* 266 */         value15 = (java.lang.String) funcResult4;
/* 267 */       }
/* 268 */
/* 269 */     }
/* 270 */     isNull15 = value15 == null;
/* 271 */     boolean isNull14 = isNull15;
/* 272 */     final UTF8String value14 = isNull14 ? null : org.apache.spark.unsafe.types.UTF8String.fromString(value15);
/* 273 */     isNull14 = value14 == null;
/* 274 */     if (isNull14) {
/* 275 */       values[0] = null;
/* 276 */     } else {
/* 277 */       values[0] = value14;
/* 278 */     }
/* 279 */   }
/* 280 */

@kiszk
Copy link
Member

kiszk commented Nov 4, 2016

Thanks, good to see generated code for ease of understanding.
While a local variable value4 at line 239 cannot be accessed before this PR, an instance variable classChildVar1 at line 306 can be accessed within this class after this PR.

(childVar, classChildVar)
}.unzip

val initClassChildrenVars = classChildrenVars.zipWithIndex.map { case (classChildrenVar, i) =>
Copy link
Contributor

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?

Copy link
Member Author

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.

@SparkQA
Copy link

SparkQA commented Nov 5, 2016

Test build #68173 has finished for PR 15693 at commit 448abfa.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member Author

viirya commented Nov 7, 2016

ping @hvanhovell Please let me know if you have other more comments. Thanks.

@SparkQA
Copy link

SparkQA commented Nov 7, 2016

Test build #68269 has finished for PR 15693 at commit 8b71061.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

asfgit pushed a commit that referenced this pull request Nov 7, 2016
…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]>
@hvanhovell
Copy link
Contributor

LGTM. Merging to master/2.1. Thanks!

Could you open a backport for 2.0?

@asfgit asfgit closed this in a814eea Nov 7, 2016
@viirya
Copy link
Member Author

viirya commented Nov 7, 2016

@hvanhovell OK. I will open a backport later.

asfgit pushed a commit that referenced this pull request Nov 10, 2016
## 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]>
asfgit pushed a commit that referenced this pull request Nov 10, 2016
## 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.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…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.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
## 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.
@viirya viirya deleted the fix-codege-compilation-error branch December 27, 2023 18:34
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants