Skip to content

Commit

Permalink
Use getConstructors() instead of try-catch to find right constructor.
Browse files Browse the repository at this point in the history
  • Loading branch information
JoshRosen committed Jan 26, 2015
1 parent d0d276d commit d6f3113
Showing 1 changed file with 26 additions and 14 deletions.
40 changes: 26 additions & 14 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark
import scala.language.implicitConversions

import java.io._
import java.lang.reflect.Constructor
import java.net.URI
import java.util.{Arrays, Properties, UUID}
import java.util.concurrent.atomic.AtomicInteger
Expand Down Expand Up @@ -1502,22 +1503,33 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
(fromSparkConf ++ fromEnvVar).map(_.trim).filter(_ != "")
}
for (className <- listenerClassNames) {
val listenerClass = Class.forName(className).asInstanceOf[Class[_ <: SparkListener]]
val listener = try {
listenerClass.getConstructor(classOf[SparkConf]).newInstance(conf)
} catch {
case e: NoSuchMethodException =>
try {
listenerClass.newInstance()
} catch {
case e: NoSuchMethodException =>
throw new SparkException(
s"$listenerClass did not have a zero-argument constructor or a" +
" single-argument constructor that accepts SparkConf (is it a nested Scala class?)")
}
val constructors = {
val listenerClass = Class.forName(className)
listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]]
}
val constructorTakingSparkConf = constructors.find { c =>
c.getParameterTypes.sameElements(Array(classOf[SparkConf]))
}
lazy val zeroArgumentConstructor = constructors.find { c =>
c.getParameterTypes.isEmpty
}
val listener: SparkListener = {
if (constructorTakingSparkConf.isDefined) {
constructorTakingSparkConf.get.newInstance(conf)
} else if (zeroArgumentConstructor.isDefined) {
zeroArgumentConstructor.get.newInstance()
} else {
throw new SparkException(
s"$className did not have a zero-argument constructor or a" +
" single-argument constructor that accepts SparkConf. Note: if the class is" +
" defined inside of another Scala class, then its constructors may accept an" +
" implicit parameter that references the enclosing class; in this case, you must" +
" define the listener as a top-level class in order to prevent this extra" +
" parameter from breaking Spark's ability to find a valid constructor.")
}
}
listenerBus.addListener(listener)
logInfo(s"Registered listener $listenerClass")
logInfo(s"Registered listener $className")
}
} catch {
case e: Exception =>
Expand Down

0 comments on commit d6f3113

Please sign in to comment.