[
https://issues.apache.org/jira/browse/SPARK-23598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16391373#comment-16391373
]
Marco Gaido commented on SPARK-23598:
-------------------------------------
[~dvogelbacher] the parameter you are talking about is taken in account only
when we split expressions and it is not done in HashAggregateExec. I haven't
been able to reproduce this. If you can provide a sample to reproduce this, it
would be very helpful.
> 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: [email protected]
For additional commands, e-mail: [email protected]