You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by gatorsmile <gi...@git.apache.org> on 2017/10/05 18:28:36 UTC

[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

GitHub user gatorsmile opened a pull request:

    https://github.com/apache/spark/pull/19440

    [SPARK-21871][SQL] Fix infinite loop when bytecode size is larger than spark.sql.codegen.hugeMethodLimit

    ## What changes were proposed in this pull request?
    When exceeding `spark.sql.codegen.hugeMethodLimit`, the runtime fallbacks to the Volcano iterator solution. This could cause an infinite loop when `FileSourceScanExec` can use the columnar batch to read the data. This PR is to fix the issue.
    
    ## How was this patch tested?
    Added a test

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/gatorsmile/spark testt

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19440.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19440
    
----
commit 6e2f53238856942e40a3301108f37a3a5cc17bca
Author: gatorsmile <ga...@gmail.com>
Date:   2017-10-05T18:20:48Z

    fix.

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82494/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    **[Test build #82485 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82485/testReport)** for PR 19440 at commit [`473bbf0`](https://github.com/apache/spark/commit/473bbf0057f50ca14bd3fe23a433d68e7b9c1c48).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    **[Test build #82494 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82494/testReport)** for PR 19440 at commit [`b8eb6a0`](https://github.com/apache/spark/commit/b8eb6a0e45ceb9592fbbf32a236aa17cd3e5dac0).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    LGTM except two minor comments.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    **[Test build #82485 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82485/testReport)** for PR 19440 at commit [`473bbf0`](https://github.com/apache/spark/commit/473bbf0057f50ca14bd3fe23a433d68e7b9c1c48).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    cc @maropu @rednaxelafx 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    Thanks! Merged to master.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19440#discussion_r143091744
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ---
    @@ -185,4 +185,22 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext {
         val (_, maxCodeSize2) = CodeGenerator.compile(codeWithLongFunctions)
         assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get)
       }
    +
    +  test("returning batch for wide table") {
    +    import testImplicits._
    +    withTempPath { dir =>
    +      val path = dir.getCanonicalPath
    +      val df = spark.range(10).select(Seq.tabulate(201) {i => ('id + i).as(s"c$i")} : _*)
    +      df.write.mode(SaveMode.Overwrite).parquet(path)
    +
    +      withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "202",
    +        SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "8000") {
    +        // donot return batch, because whole stage codegen is disabled for wide table (>202 columns)
    --- End diff --
    
    Is this comment wrong or I misunderstand it? Looks like it returns batch as it asserts `supportsBatch`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19440#discussion_r143091966
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ---
    @@ -185,4 +185,22 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext {
         val (_, maxCodeSize2) = CodeGenerator.compile(codeWithLongFunctions)
         assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get)
       }
    +
    +  test("returning batch for wide table") {
    +    import testImplicits._
    +    withTempPath { dir =>
    +      val path = dir.getCanonicalPath
    +      val df = spark.range(10).select(Seq.tabulate(201) {i => ('id + i).as(s"c$i")} : _*)
    +      df.write.mode(SaveMode.Overwrite).parquet(path)
    +
    +      withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "202",
    +        SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "8000") {
    +        // donot return batch, because whole stage codegen is disabled for wide table (>202 columns)
    --- End diff --
    
    this is copied and pasted. will fix it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19440#discussion_r143089714
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala ---
    @@ -392,12 +392,16 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co
     
         // Check if compiled code has a too large function
         if (maxCodeSize > sqlContext.conf.hugeMethodLimit) {
    -      logWarning(s"Found too long generated codes and JIT optimization might not work: " +
    -        s"the bytecode size was $maxCodeSize, this value went over the limit " +
    +      logInfo(s"Found too long generated codes and JIT optimization might not work: " +
    +        s"the bytecode size ($maxCodeSize) is above the limit " +
             s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " +
             s"for this plan. To avoid this, you can raise the limit " +
    -        s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}:\n$treeString")
    -      return child.execute()
    +        s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString")
    +      child match {
    +        // For batch file source scan, we should continue executing it
    +        case f: FileSourceScanExec if f.supportsBatch => // do nothing
    --- End diff --
    
    If we do it in `FileSourceScanExec `, we are unable to know which causes the fallback. Now, we have at least two reasons that trigger the fallback. 
    
    Ideally, we should not call `WholeStageCodegenExec ` in `doExecute`. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19440#discussion_r143094785
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala ---
    @@ -392,12 +392,16 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co
     
         // Check if compiled code has a too large function
         if (maxCodeSize > sqlContext.conf.hugeMethodLimit) {
    -      logWarning(s"Found too long generated codes and JIT optimization might not work: " +
    -        s"the bytecode size was $maxCodeSize, this value went over the limit " +
    +      logInfo(s"Found too long generated codes and JIT optimization might not work: " +
    +        s"the bytecode size ($maxCodeSize) is above the limit " +
             s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " +
             s"for this plan. To avoid this, you can raise the limit " +
    -        s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}:\n$treeString")
    -      return child.execute()
    +        s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString")
    +      child match {
    +        // For batch file source scan, we should continue executing it
    +        case f: FileSourceScanExec if f.supportsBatch => // do nothing
    --- End diff --
    
    yea, I totally agree that we need to refactor this in future. Anyway, it's ok for now.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    Thanks for pining! LGTM except for one comment.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by ajithme <gi...@git.apache.org>.
Github user ajithme commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    @gatorsmile i have a question, should also be handled from other execs.? For example, like https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala#L306 and https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala#L111


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19440#discussion_r143092189
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala ---
    @@ -392,12 +392,16 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co
     
         // Check if compiled code has a too large function
         if (maxCodeSize > sqlContext.conf.hugeMethodLimit) {
    -      logWarning(s"Found too long generated codes and JIT optimization might not work: " +
    -        s"the bytecode size was $maxCodeSize, this value went over the limit " +
    +      logInfo(s"Found too long generated codes and JIT optimization might not work: " +
    +        s"the bytecode size ($maxCodeSize) is above the limit " +
             s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " +
             s"for this plan. To avoid this, you can raise the limit " +
    -        s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}:\n$treeString")
    -      return child.execute()
    +        s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString")
    +      child match {
    +        // For batch file source scan, we should continue executing it
    --- End diff --
    
    It's better to explain why we should continue it. Otherwise later readers may not understand it immediately.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19440#discussion_r143086096
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala ---
    @@ -392,12 +392,16 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co
     
         // Check if compiled code has a too large function
         if (maxCodeSize > sqlContext.conf.hugeMethodLimit) {
    -      logWarning(s"Found too long generated codes and JIT optimization might not work: " +
    -        s"the bytecode size was $maxCodeSize, this value went over the limit " +
    +      logInfo(s"Found too long generated codes and JIT optimization might not work: " +
    +        s"the bytecode size ($maxCodeSize) is above the limit " +
             s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " +
             s"for this plan. To avoid this, you can raise the limit " +
    -        s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}:\n$treeString")
    -      return child.execute()
    +        s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString")
    +      child match {
    +        // For batch file source scan, we should continue executing it
    +        case f: FileSourceScanExec if f.supportsBatch => // do nothing
    --- End diff --
    
    I feel a little weird `WholeStageCodegenExec` has specific error handling for each spark plan. Could we handle this error inside `FileSourceScanExec`? For example, how about checking if `parent.isInstanceOf[WholeStageCodegenExec]` in `FileSourceScanExec`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82485/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19440: [SPARK-21871][SQL] Fix infinite loop when bytecode size ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19440
  
    **[Test build #82494 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82494/testReport)** for PR 19440 at commit [`b8eb6a0`](https://github.com/apache/spark/commit/b8eb6a0e45ceb9592fbbf32a236aa17cd3e5dac0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19440: [SPARK-21871][SQL] Fix infinite loop when bytecod...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/19440


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org