Skip to content

Commit a539fae

Browse files
committed
[SPARK-15622] [SQL] Wrap the parent classloader of Janino's classloader in the ParentClassLoader.
1 parent 4b88067 commit a539fae

File tree

2 files changed

+14
-10
lines changed

2 files changed

+14
-10
lines changed

repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -79,13 +79,7 @@ class ExecutorClassLoader(
7979
case e: ClassNotFoundException =>
8080
val classOption = findClassLocally(name)
8181
classOption match {
82-
case None =>
83-
// If this class has a cause, it will break the internal assumption of Janino
84-
// (the compiler used for Spark SQL code-gen).
85-
// See org.codehaus.janino.ClassLoaderIClassLoader's findIClass, you will see
86-
// its behavior will be changed if there is a cause and the compilation
87-
// of generated class will fail.
88-
throw new ClassNotFoundException(name)
82+
case None => throw new ClassNotFoundException(name, e)
8983
case Some(a) => a
9084
}
9185
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,10 +19,10 @@ package org.apache.spark.sql.catalyst.expressions.codegen
1919

2020
import scala.collection.mutable
2121
import scala.collection.mutable.ArrayBuffer
22-
import scala.language.existentials
2322

2423
import com.google.common.cache.{CacheBuilder, CacheLoader}
2524
import org.codehaus.janino.ClassBodyEvaluator
25+
import scala.language.existentials
2626

2727
import org.apache.spark.internal.Logging
2828
import org.apache.spark.sql.catalyst.InternalRow
@@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
3131
import org.apache.spark.sql.types._
3232
import org.apache.spark.unsafe.Platform
3333
import org.apache.spark.unsafe.types._
34-
import org.apache.spark.util.Utils
34+
import org.apache.spark.util.{ParentClassLoader, Utils}
3535

3636
/**
3737
* Java source for evaluating an [[Expression]] given a [[InternalRow]] of input.
@@ -806,7 +806,17 @@ object CodeGenerator extends Logging {
806806
*/
807807
private[this] def doCompile(code: CodeAndComment): GeneratedClass = {
808808
val evaluator = new ClassBodyEvaluator()
809-
evaluator.setParentClassLoader(Utils.getContextOrSparkClassLoader)
809+
810+
// A special classloader used to wrap the actual parent classloader of
811+
// [[org.codehaus.janino.ClassBodyEvaluator]] (see CodeGenerator.doCompile). This classloader
812+
// does not throw a ClassNotFoundException with a cause set (i.e. exception.getCause returns
813+
// a null). This classloader is needed because janino will throw the exception directly if
814+
// the parent classloader throws a ClassNotFoundException with cause set instead of trying to
815+
// find other possible classes (see org.codehaus.janinoClassLoaderIClassLoader's
816+
// findIClass method). Please also see https://issues.apache.org/jira/browse/SPARK-15622 and
817+
// https://issues.apache.org/jira/browse/SPARK-11636.
818+
val parentClassLoader = new ParentClassLoader(Utils.getContextOrSparkClassLoader)
819+
evaluator.setParentClassLoader(parentClassLoader)
810820
// Cannot be under package codegen, or fail with java.lang.InstantiationException
811821
evaluator.setClassName("org.apache.spark.sql.catalyst.expressions.GeneratedClass")
812822
evaluator.setDefaultImports(Array(

0 commit comments

Comments
 (0)