[ https://issues.apache.org/jira/browse/SPARK-23598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16391709#comment-16391709 ]
David Vogelbacher edited comment on SPARK-23598 at 3/8/18 6:41 PM: ------------------------------------------------------------------- [~mgaido] {{HashAggregateExec}} calls {{addNewFunction}}, which calls {{addNewFunctionInternal}} which uses that flag and checks if the current size is bigger than {{GENERATED_CLASS_SIZE_THRESHOLD}} ([see|https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala#L478]) I just compiled develop with {{GENERATED_CLASS_SIZE_THRESHOLD}} set to -1 and was able to reproduce (cc [~hvanhovell]) . I applied the following diff before compiling: {noformat} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 793824b0b0..7fad817d89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1167,7 +1167,7 @@ object CodeGenerator extends Logging { // limit, 65,536. We cannot know how many constants will be inserted for a class, so we use a // threshold of 1000k bytes to determine when a function should be inlined to a private, inner // class. - final val GENERATED_CLASS_SIZE_THRESHOLD = 1000000 + final val GENERATED_CLASS_SIZE_THRESHOLD = -1 // This is the threshold for the number of global variables, whose types are primitive type or // complex type (e.g. more than one-dimensional array), that will be placed at the outer class (END) {noformat} Then, I executed a simple groupBy-Aggregate in the spark-shell and got the same error: {noformat} ➜ spark git:(master) ✗ ./bin/spark-shell 18/03/08 18:30:24 WARN Utils: Your hostname, dvogelbac resolves to a loopback address: 127.0.0.1; using 10.111.11.111 instead (on interface en0) 18/03/08 18:30:24 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address 18/03/08 18:30:24 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). Spark context Web UI available at http://10.111.11.111:4040 Spark context available as 'sc' (master = local[*], app id = local-1520533829643). Spark session available as 'spark'. Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.4.0-SNAPSHOT /_/ Using Scala version 2.11.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_121) Type in expressions to have them evaluated. Type :help for more information. scala> spark.conf.set("spark.sql.codegen.wholeStage", true) scala> val df_pet_age = Seq( | (8, "bat"), | (5, "bat"), | (15, "bat"), | (30, "mouse"), | (15, "mouse"), | (23, "mouse"), | (8, "horse"), | (-5, "horse") | ).toDF("age", "name") df_pet_age: org.apache.spark.sql.DataFrame = [age: int, name: string] scala> df_pet_age.groupBy("name").avg("age").show() 18/03/08 18:31:20 ERROR Executor: Exception in task 1.0 in stage 0.0 (TID 1) java.lang.IllegalAccessError: tried to access method org.apache.spark.sql.execution.BufferedRowIterator.stopEarly()Z from class org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1 at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1.agg_doAggregateWithKeys$(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:616) at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) at org.apache.spark.scheduler.Task.run(Task.scala:109) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 18/03/08 18:31:20 ERROR Executor: Exception in task 5.0 in stage 0.0 (TID 5) {noformat} It is actually failing trying to invoke a different method in {{BufferedRowIterator}} now ({{stopEarly}}), but it is the same problem. That method is also declared {{protected}} and can't be accessed. was (Author: dvogelbacher): [~mgaido] {{HashAggregateExec}} calls {{addNewFunction}}, which calls {{addNewFunctionInternal}} which uses that flag and checks if the current size is bigger than {{GENERATED_CLASS_SIZE_THRESHOLD}} ([see|https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala#L478]) I just compiled develop with {{GENERATED_CLASS_SIZE_THRESHOLD}} set to -1 and was able to reproduce (cc [~hvanhovell]) . I applied the following diff before compiling: {noformat} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 793824b0b0..7fad817d89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1167,7 +1167,7 @@ object CodeGenerator extends Logging { // limit, 65,536. We cannot know how many constants will be inserted for a class, so we use a // threshold of 1000k bytes to determine when a function should be inlined to a private, inner // class. - final val GENERATED_CLASS_SIZE_THRESHOLD = 1000000 + final val GENERATED_CLASS_SIZE_THRESHOLD = -1 // This is the threshold for the number of global variables, whose types are primitive type or // complex type (e.g. more than one-dimensional array), that will be placed at the outer class (END) {noformat} Then, I executed a simple groupBy-Aggregate in the spark-shell and got the same error: {noformat} ➜ spark git:(master) ✗ ./bin/spark-shell 18/03/08 18:30:24 WARN Utils: Your hostname, dvogelbac56-mac resolves to a loopback address: 127.0.0.1; using 10.224.86.161 instead (on interface en0) 18/03/08 18:30:24 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address 18/03/08 18:30:24 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). Spark context Web UI available at http://10.224.86.161:4040 Spark context available as 'sc' (master = local[*], app id = local-1520533829643). Spark session available as 'spark'. Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.4.0-SNAPSHOT /_/ Using Scala version 2.11.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_121) Type in expressions to have them evaluated. Type :help for more information. scala> spark.conf.set("spark.sql.codegen.wholeStage", true) scala> val df_pet_age = Seq( | (8, "bat"), | (5, "bat"), | (15, "bat"), | (30, "mouse"), | (15, "mouse"), | (23, "mouse"), | (8, "horse"), | (-5, "horse") | ).toDF("age", "name") df_pet_age: org.apache.spark.sql.DataFrame = [age: int, name: string] scala> df_pet_age.groupBy("name").avg("age").show() 18/03/08 18:31:20 ERROR Executor: Exception in task 1.0 in stage 0.0 (TID 1) java.lang.IllegalAccessError: tried to access method org.apache.spark.sql.execution.BufferedRowIterator.stopEarly()Z from class org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1 at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1.agg_doAggregateWithKeys$(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:616) at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) at org.apache.spark.scheduler.Task.run(Task.scala:109) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 18/03/08 18:31:20 ERROR Executor: Exception in task 5.0 in stage 0.0 (TID 5) {noformat} It is actually failing trying to invoke a different method in {{BufferedRowIterator}} now ({{stopEarly}}), but it is the same problem. That method is also declared {{protected}} and can't be accessed. > WholeStageCodegen can lead to IllegalAccessError calling append for > HashAggregateExec > -------------------------------------------------------------------------------------- > > Key: SPARK-23598 > URL: https://issues.apache.org/jira/browse/SPARK-23598 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 2.4.0 > Reporter: David Vogelbacher > Priority: Major > > Got the following stacktrace for a large QueryPlan using WholeStageCodeGen: > {noformat} > java.lang.IllegalAccessError: tried to access method > org.apache.spark.sql.execution.BufferedRowIterator.append(Lorg/apache/spark/sql/catalyst/InternalRow;)V > from class > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage7$agg_NestedClass > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage7$agg_NestedClass.agg_doAggregateWithKeysOutput$(Unknown > Source) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage7.processNext(Unknown > Source) > at > org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) > at > org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) > at org.apache.spark.scheduler.Task.run(Task.scala:109) > at > org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345){noformat} > After disabling codegen, everything works. > The root cause seems to be that we are trying to call the protected _append_ > method of > [BufferedRowIterator|https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java#L68] > from an inner-class of a sub-class that is loaded by a different > class-loader (after codegen compilation). > [https://docs.oracle.com/javase/specs/jvms/se7/html/jvms-5.html#jvms-5.4.4] > states that a protected method _R_ can be accessed only if one of the > following two conditions is fulfilled: > # R is protected and is declared in a class C, and D is either a subclass of > C or C itself. Furthermore, if R is not static, then the symbolic reference > to R must contain a symbolic reference to a class T, such that T is either a > subclass of D, a superclass of D, or D itself. > # R is either protected or has default access (that is, neither public nor > protected nor private), and is declared by a class in the same run-time > package as D. > 2.) doesn't apply as we have loaded the class with a different class loader > (and are in a different package) and 1.) doesn't apply because we are > apparently trying to call the method from an inner class of a subclass of > _BufferedRowIterator_. > Looking at the Code path of _WholeStageCodeGen_, the following happens: > # In > [WholeStageCodeGen|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala#L527], > we create the subclass of _BufferedRowIterator_, along with a _processNext_ > method for processing the output of the child plan. > # In the child, which is a > [HashAggregateExec|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L517], > we create the method which shows up at the top of the stack trace (called > _doAggregateWithKeysOutput_ ) > # We add this method to the compiled code invoking _addNewFunction_ of > [CodeGenerator|https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala#L460] > In the generated function body we call the _append_ method.| > Now, the _addNewFunction_ method states that: > {noformat} > If the code for the `OuterClass` grows too large, the function will be > inlined into a new private, inner class > {noformat} > This indeed seems to happen: the _doAggregateWithKeysOutput_ method is put > into a new private inner class. Thus, it doesn't have access to the protected > _append_ method anymore but still tries to call it, which results in the > _IllegalAccessError._ > Possible fixes: > * Pass in the _inlineToOuterClass_ flag when invoking the _addNewFunction_ > * Make the _append_ method public > * Re-declare the _append_ method in the generated subclass (just invoking > _super_). This way, inner classes should have access to it. > -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org