Skip to content

Commit d6f3113

Browse files
committed
Use getConstructors() instead of try-catch to find right constructor.
1 parent d0d276d commit d6f3113

File tree

1 file changed

+26
-14
lines changed

1 file changed

+26
-14
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 26 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark
2020
import scala.language.implicitConversions
2121

2222
import java.io._
23+
import java.lang.reflect.Constructor
2324
import java.net.URI
2425
import java.util.{Arrays, Properties, UUID}
2526
import java.util.concurrent.atomic.AtomicInteger
@@ -1502,22 +1503,33 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
15021503
(fromSparkConf ++ fromEnvVar).map(_.trim).filter(_ != "")
15031504
}
15041505
for (className <- listenerClassNames) {
1505-
val listenerClass = Class.forName(className).asInstanceOf[Class[_ <: SparkListener]]
1506-
val listener = try {
1507-
listenerClass.getConstructor(classOf[SparkConf]).newInstance(conf)
1508-
} catch {
1509-
case e: NoSuchMethodException =>
1510-
try {
1511-
listenerClass.newInstance()
1512-
} catch {
1513-
case e: NoSuchMethodException =>
1514-
throw new SparkException(
1515-
s"$listenerClass did not have a zero-argument constructor or a" +
1516-
" single-argument constructor that accepts SparkConf (is it a nested Scala class?)")
1517-
}
1506+
val constructors = {
1507+
val listenerClass = Class.forName(className)
1508+
listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]]
1509+
}
1510+
val constructorTakingSparkConf = constructors.find { c =>
1511+
c.getParameterTypes.sameElements(Array(classOf[SparkConf]))
1512+
}
1513+
lazy val zeroArgumentConstructor = constructors.find { c =>
1514+
c.getParameterTypes.isEmpty
1515+
}
1516+
val listener: SparkListener = {
1517+
if (constructorTakingSparkConf.isDefined) {
1518+
constructorTakingSparkConf.get.newInstance(conf)
1519+
} else if (zeroArgumentConstructor.isDefined) {
1520+
zeroArgumentConstructor.get.newInstance()
1521+
} else {
1522+
throw new SparkException(
1523+
s"$className did not have a zero-argument constructor or a" +
1524+
" single-argument constructor that accepts SparkConf. Note: if the class is" +
1525+
" defined inside of another Scala class, then its constructors may accept an" +
1526+
" implicit parameter that references the enclosing class; in this case, you must" +
1527+
" define the listener as a top-level class in order to prevent this extra" +
1528+
" parameter from breaking Spark's ability to find a valid constructor.")
1529+
}
15181530
}
15191531
listenerBus.addListener(listener)
1520-
logInfo(s"Registered listener $listenerClass")
1532+
logInfo(s"Registered listener $className")
15211533
}
15221534
} catch {
15231535
case e: Exception =>

0 commit comments

Comments
 (0)