You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2015/07/15 01:08:22 UTC

spark git commit: [SPARK-8962] Add Scalastyle rule to ban direct use of Class.forName; fix existing uses

Repository: spark
Updated Branches:
  refs/heads/master 740b034f1 -> 11e5c3728


[SPARK-8962] Add Scalastyle rule to ban direct use of Class.forName; fix existing uses

This pull request adds a Scalastyle regex rule which fails the style check if `Class.forName` is used directly.  `Class.forName` always loads classes from the default / system classloader, but in a majority of cases, we should be using Spark's own `Utils.classForName` instead, which tries to load classes from the current thread's context classloader and falls back to the classloader which loaded Spark when the context classloader is not defined.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7350)
<!-- Reviewable:end -->

Author: Josh Rosen <jo...@databricks.com>

Closes #7350 from JoshRosen/ban-Class.forName and squashes the following commits:

e3e96f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
c0b7885 [Josh Rosen] Hopefully fix the last two cases
d707ba7 [Josh Rosen] Fix uses of Class.forName that I missed in my first cleanup pass
046470d [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
62882ee [Josh Rosen] Fix uses of Class.forName or add exclusion.
d9abade [Josh Rosen] Add stylechecker rule to ban uses of Class.forName


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/11e5c372
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/11e5c372
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/11e5c372

Branch: refs/heads/master
Commit: 11e5c372862ec00e57460b37ccfee51c6d93c5f7
Parents: 740b034
Author: Josh Rosen <jo...@databricks.com>
Authored: Tue Jul 14 16:08:17 2015 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Tue Jul 14 16:08:17 2015 -0700

----------------------------------------------------------------------
 .../src/main/scala/org/apache/spark/Logging.scala |  2 +-
 .../scala/org/apache/spark/SparkContext.scala     | 11 +++++------
 .../main/scala/org/apache/spark/SparkEnv.scala    |  2 +-
 .../org/apache/spark/api/r/RBackendHandler.scala  | 18 ++----------------
 .../apache/spark/broadcast/BroadcastManager.scala |  3 ++-
 .../org/apache/spark/deploy/SparkHadoopUtil.scala |  4 ++--
 .../org/apache/spark/deploy/SparkSubmit.scala     |  2 +-
 .../spark/deploy/SparkSubmitArguments.scala       |  2 +-
 .../spark/deploy/history/HistoryServer.scala      |  2 +-
 .../org/apache/spark/deploy/master/Master.scala   |  2 +-
 .../deploy/rest/SubmitRestProtocolMessage.scala   |  2 +-
 .../spark/deploy/worker/DriverWrapper.scala       |  2 +-
 .../spark/deploy/worker/WorkerArguments.scala     |  2 ++
 .../org/apache/spark/executor/Executor.scala      |  2 +-
 .../org/apache/spark/io/CompressionCodec.scala    |  3 +--
 .../spark/mapred/SparkHadoopMapRedUtil.scala      |  5 +++--
 .../mapreduce/SparkHadoopMapReduceUtil.scala      |  9 +++++----
 .../org/apache/spark/metrics/MetricsSystem.scala  |  6 ++++--
 .../scala/org/apache/spark/rdd/HadoopRDD.scala    |  6 +++---
 .../main/scala/org/apache/spark/rpc/RpcEnv.scala  |  3 +--
 .../apache/spark/serializer/JavaSerializer.scala  |  5 ++++-
 .../apache/spark/serializer/KryoSerializer.scala  |  2 ++
 .../spark/serializer/SerializationDebugger.scala  |  2 ++
 .../apache/spark/storage/ExternalBlockStore.scala |  2 +-
 .../org/apache/spark/util/ClosureCleaner.scala    |  2 ++
 .../org/apache/spark/util/SizeEstimator.scala     |  2 ++
 .../main/scala/org/apache/spark/util/Utils.scala  | 11 +++++++++--
 .../test/scala/org/apache/spark/FileSuite.scala   |  2 ++
 .../SparkContextSchedulerCreationSuite.scala      |  3 ++-
 .../apache/spark/deploy/SparkSubmitSuite.scala    |  4 ++--
 .../scala/org/apache/spark/rdd/JdbcRDDSuite.scala |  3 ++-
 .../KryoSerializerDistributedSuite.scala          |  2 ++
 .../spark/util/MutableURLClassLoaderSuite.scala   |  2 ++
 .../spark/streaming/flume/sink/Logging.scala      |  2 ++
 .../apache/spark/graphx/util/BytecodeUtils.scala  |  2 +-
 .../scala/org/apache/spark/repl/SparkIMain.scala  |  2 ++
 scalastyle-config.xml                             | 11 +++++++++++
 .../org/apache/spark/sql/types/DataType.scala     |  3 ++-
 .../scala/org/apache/spark/sql/SQLContext.scala   |  3 +--
 .../spark/sql/parquet/ParquetRelation.scala       |  7 ++++---
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala     |  3 ++-
 .../apache/spark/sql/jdbc/JDBCWriteSuite.scala    |  3 ++-
 .../thriftserver/HiveThriftServer2Suites.scala    |  2 +-
 .../org/apache/spark/sql/hive/TableReader.scala   |  4 +---
 .../spark/sql/hive/client/ClientWrapper.scala     |  9 ++++-----
 .../spark/sql/hive/HiveSparkSubmitSuite.scala     |  8 ++++----
 .../spark/streaming/scheduler/JobGenerator.scala  |  6 +++---
 .../apache/spark/tools/GenerateMIMAIgnore.scala   |  2 ++
 .../org/apache/spark/deploy/yarn/Client.scala     |  4 ++--
 49 files changed, 117 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/Logging.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index 87ab099..f059881 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -159,7 +159,7 @@ private object Logging {
   try {
     // We use reflection here to handle the case where users remove the
     // slf4j-to-jul bridge order to route their logs to JUL.
-    val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler")
+    val bridgeClass = Utils.classForName("org.slf4j.bridge.SLF4JBridgeHandler")
     bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null)
     val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean]
     if (!installed) {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 82704b1..bd1cc33 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1968,7 +1968,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
       for (className <- listenerClassNames) {
         // Use reflection to find the right constructor
         val constructors = {
-          val listenerClass = Class.forName(className)
+          val listenerClass = Utils.classForName(className)
           listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]]
         }
         val constructorTakingSparkConf = constructors.find { c =>
@@ -2503,7 +2503,7 @@ object SparkContext extends Logging {
             "\"yarn-standalone\" is deprecated as of Spark 1.0. Use \"yarn-cluster\" instead.")
         }
         val scheduler = try {
-          val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
+          val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
           val cons = clazz.getConstructor(classOf[SparkContext])
           cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
         } catch {
@@ -2515,7 +2515,7 @@ object SparkContext extends Logging {
         }
         val backend = try {
           val clazz =
-            Class.forName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend")
+            Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend")
           val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
           cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
         } catch {
@@ -2528,8 +2528,7 @@ object SparkContext extends Logging {
 
       case "yarn-client" =>
         val scheduler = try {
-          val clazz =
-            Class.forName("org.apache.spark.scheduler.cluster.YarnScheduler")
+          val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnScheduler")
           val cons = clazz.getConstructor(classOf[SparkContext])
           cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
 
@@ -2541,7 +2540,7 @@ object SparkContext extends Logging {
 
         val backend = try {
           val clazz =
-            Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend")
+            Utils.classForName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend")
           val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
           cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
         } catch {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/SparkEnv.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index d18fc59..adfece4 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -261,7 +261,7 @@ object SparkEnv extends Logging {
 
     // Create an instance of the class with the given name, possibly initializing it with our conf
     def instantiateClass[T](className: String): T = {
-      val cls = Class.forName(className, true, Utils.getContextOrSparkClassLoader)
+      val cls = Utils.classForName(className)
       // Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just
       // SparkConf, then one taking no arguments
       try {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala
index 4b8f7fe..9658e9a 100644
--- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala
@@ -26,6 +26,7 @@ import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler}
 
 import org.apache.spark.Logging
 import org.apache.spark.api.r.SerDe._
+import org.apache.spark.util.Utils
 
 /**
  * Handler for RBackend
@@ -88,21 +89,6 @@ private[r] class RBackendHandler(server: RBackend)
     ctx.close()
   }
 
-  // Looks up a class given a class name. This function first checks the
-  // current class loader and if a class is not found, it looks up the class
-  // in the context class loader. Address [SPARK-5185]
-  def getStaticClass(objId: String): Class[_] = {
-    try {
-      val clsCurrent = Class.forName(objId)
-      clsCurrent
-    } catch {
-      // Use contextLoader if we can't find the JAR in the system class loader
-      case e: ClassNotFoundException =>
-        val clsContext = Class.forName(objId, true, Thread.currentThread().getContextClassLoader)
-        clsContext
-      }
-    }
-
   def handleMethodCall(
       isStatic: Boolean,
       objId: String,
@@ -113,7 +99,7 @@ private[r] class RBackendHandler(server: RBackend)
     var obj: Object = null
     try {
       val cls = if (isStatic) {
-        getStaticClass(objId)
+        Utils.classForName(objId)
       } else {
         JVMObjectTracker.get(objId) match {
           case None => throw new IllegalArgumentException("Object not found " + objId)

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
index 685313a..fac6666 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
@@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicLong
 import scala.reflect.ClassTag
 
 import org.apache.spark._
+import org.apache.spark.util.Utils
 
 private[spark] class BroadcastManager(
     val isDriver: Boolean,
@@ -42,7 +43,7 @@ private[spark] class BroadcastManager(
           conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory")
 
         broadcastFactory =
-          Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
+          Utils.classForName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
 
         // Initialize appropriate BroadcastFactory and BroadcastObject
         broadcastFactory.initialize(isDriver, conf, securityManager)

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 6d14590..9f94118 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -178,7 +178,7 @@ class SparkHadoopUtil extends Logging {
 
   private def getFileSystemThreadStatisticsMethod(methodName: String): Method = {
     val statisticsDataClass =
-      Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData")
+      Utils.classForName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData")
     statisticsDataClass.getDeclaredMethod(methodName)
   }
 
@@ -356,7 +356,7 @@ object SparkHadoopUtil {
         System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
     if (yarnMode) {
       try {
-        Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
+        Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
           .newInstance()
           .asInstanceOf[SparkHadoopUtil]
       } catch {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 7089a7e..036cb6e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -624,7 +624,7 @@ object SparkSubmit {
     var mainClass: Class[_] = null
 
     try {
-      mainClass = Class.forName(childMainClass, true, loader)
+      mainClass = Utils.classForName(childMainClass)
     } catch {
       case e: ClassNotFoundException =>
         e.printStackTrace(printStream)

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index ebb39c3..b371007 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -576,7 +576,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
       System.setSecurityManager(sm)
 
       try {
-        Class.forName(mainClass).getMethod("main", classOf[Array[String]])
+        Utils.classForName(mainClass).getMethod("main", classOf[Array[String]])
           .invoke(null, Array(HELP))
       } catch {
         case e: InvocationTargetException =>

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index 10638af..a076a9c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -228,7 +228,7 @@ object HistoryServer extends Logging {
 
     val providerName = conf.getOption("spark.history.provider")
       .getOrElse(classOf[FsHistoryProvider].getName())
-    val provider = Class.forName(providerName)
+    val provider = Utils.classForName(providerName)
       .getConstructor(classOf[SparkConf])
       .newInstance(conf)
       .asInstanceOf[ApplicationHistoryProvider]

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 4807076..245b047 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -172,7 +172,7 @@ private[master] class Master(
           new FileSystemRecoveryModeFactory(conf, SerializationExtension(actorSystem))
         (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this))
       case "CUSTOM" =>
-        val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory"))
+        val clazz = Utils.classForName(conf.get("spark.deploy.recoveryMode.factory"))
         val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serialization])
           .newInstance(conf, SerializationExtension(actorSystem))
           .asInstanceOf[StandaloneRecoveryModeFactory]

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala
index e6615a3..ef5a7e3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala
@@ -128,7 +128,7 @@ private[spark] object SubmitRestProtocolMessage {
    */
   def fromJson(json: String): SubmitRestProtocolMessage = {
     val className = parseAction(json)
-    val clazz = Class.forName(packagePrefix + "." + className)
+    val clazz = Utils.classForName(packagePrefix + "." + className)
       .asSubclass[SubmitRestProtocolMessage](classOf[SubmitRestProtocolMessage])
     fromJson(json, clazz)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index 2d6be30..6799f78 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -53,7 +53,7 @@ object DriverWrapper {
         Thread.currentThread.setContextClassLoader(loader)
 
         // Delegate to supplied main class
-        val clazz = Class.forName(mainClass, true, loader)
+        val clazz = Utils.classForName(mainClass)
         val mainMethod = clazz.getMethod("main", classOf[Array[String]])
         mainMethod.invoke(null, extraArgs.toArray[String])
 

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index e89d076..5181142 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -149,6 +149,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
     val ibmVendor = System.getProperty("java.vendor").contains("IBM")
     var totalMb = 0
     try {
+      // scalastyle:off classforname
       val bean = ManagementFactory.getOperatingSystemMXBean()
       if (ibmVendor) {
         val beanClass = Class.forName("com.ibm.lang.management.OperatingSystemMXBean")
@@ -159,6 +160,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
         val method = beanClass.getDeclaredMethod("getTotalPhysicalMemorySize")
         totalMb = (method.invoke(bean).asInstanceOf[Long] / 1024 / 1024).toInt
       }
+      // scalastyle:on classforname
     } catch {
       case e: Exception => {
         totalMb = 2*1024

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index f7ef92b..1a02051 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -356,7 +356,7 @@ private[spark] class Executor(
       logInfo("Using REPL class URI: " + classUri)
       try {
         val _userClassPathFirst: java.lang.Boolean = userClassPathFirst
-        val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader")
+        val klass = Utils.classForName("org.apache.spark.repl.ExecutorClassLoader")
           .asInstanceOf[Class[_ <: ClassLoader]]
         val constructor = klass.getConstructor(classOf[SparkConf], classOf[String],
           classOf[ClassLoader], classOf[Boolean])

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 0d8ac1f..607d5a3 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -63,8 +63,7 @@ private[spark] object CompressionCodec {
   def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
     val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName)
     val codec = try {
-      val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader)
-        .getConstructor(classOf[SparkConf])
+      val ctor = Utils.classForName(codecClass).getConstructor(classOf[SparkConf])
       Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec])
     } catch {
       case e: ClassNotFoundException => None

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
index 818f7a4..87df427 100644
--- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
+++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
@@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter}
 
 import org.apache.spark.executor.CommitDeniedException
 import org.apache.spark.{Logging, SparkEnv, TaskContext}
+import org.apache.spark.util.{Utils => SparkUtils}
 
 private[spark]
 trait SparkHadoopMapRedUtil {
@@ -64,10 +65,10 @@ trait SparkHadoopMapRedUtil {
 
   private def firstAvailableClass(first: String, second: String): Class[_] = {
     try {
-      Class.forName(first)
+      SparkUtils.classForName(first)
     } catch {
       case e: ClassNotFoundException =>
-        Class.forName(second)
+        SparkUtils.classForName(second)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala
index 390d148..943ebcb 100644
--- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala
+++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala
@@ -21,6 +21,7 @@ import java.lang.{Boolean => JBoolean, Integer => JInteger}
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID}
+import org.apache.spark.util.Utils
 
 private[spark]
 trait SparkHadoopMapReduceUtil {
@@ -46,7 +47,7 @@ trait SparkHadoopMapReduceUtil {
       isMap: Boolean,
       taskId: Int,
       attemptId: Int): TaskAttemptID = {
-    val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID")
+    val klass = Utils.classForName("org.apache.hadoop.mapreduce.TaskAttemptID")
     try {
       // First, attempt to use the old-style constructor that takes a boolean isMap
       // (not available in YARN)
@@ -57,7 +58,7 @@ trait SparkHadoopMapReduceUtil {
     } catch {
       case exc: NoSuchMethodException => {
         // If that failed, look for the new constructor that takes a TaskType (not available in 1.x)
-        val taskTypeClass = Class.forName("org.apache.hadoop.mapreduce.TaskType")
+        val taskTypeClass = Utils.classForName("org.apache.hadoop.mapreduce.TaskType")
           .asInstanceOf[Class[Enum[_]]]
         val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke(
           taskTypeClass, if (isMap) "MAP" else "REDUCE")
@@ -71,10 +72,10 @@ trait SparkHadoopMapReduceUtil {
 
   private def firstAvailableClass(first: String, second: String): Class[_] = {
     try {
-      Class.forName(first)
+      Utils.classForName(first)
     } catch {
       case e: ClassNotFoundException =>
-        Class.forName(second)
+        Utils.classForName(second)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index ed5131c..67f64d5 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -20,6 +20,8 @@ package org.apache.spark.metrics
 import java.util.Properties
 import java.util.concurrent.TimeUnit
 
+import org.apache.spark.util.Utils
+
 import scala.collection.mutable
 
 import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry}
@@ -166,7 +168,7 @@ private[spark] class MetricsSystem private (
     sourceConfigs.foreach { kv =>
       val classPath = kv._2.getProperty("class")
       try {
-        val source = Class.forName(classPath).newInstance()
+        val source = Utils.classForName(classPath).newInstance()
         registerSource(source.asInstanceOf[Source])
       } catch {
         case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e)
@@ -182,7 +184,7 @@ private[spark] class MetricsSystem private (
       val classPath = kv._2.getProperty("class")
       if (null != classPath) {
         try {
-          val sink = Class.forName(classPath)
+          val sink = Utils.classForName(classPath)
             .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager])
             .newInstance(kv._2, registry, securityMgr)
           if (kv._1 == "servlet") {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index bee59a4..f1c1736 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -383,11 +383,11 @@ private[spark] object HadoopRDD extends Logging {
 
   private[spark] class SplitInfoReflections {
     val inputSplitWithLocationInfo =
-      Class.forName("org.apache.hadoop.mapred.InputSplitWithLocationInfo")
+      Utils.classForName("org.apache.hadoop.mapred.InputSplitWithLocationInfo")
     val getLocationInfo = inputSplitWithLocationInfo.getMethod("getLocationInfo")
-    val newInputSplit = Class.forName("org.apache.hadoop.mapreduce.InputSplit")
+    val newInputSplit = Utils.classForName("org.apache.hadoop.mapreduce.InputSplit")
     val newGetLocationInfo = newInputSplit.getMethod("getLocationInfo")
-    val splitLocationInfo = Class.forName("org.apache.hadoop.mapred.SplitLocationInfo")
+    val splitLocationInfo = Utils.classForName("org.apache.hadoop.mapred.SplitLocationInfo")
     val isInMemory = splitLocationInfo.getMethod("isInMemory")
     val getLocation = splitLocationInfo.getMethod("getLocation")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
index 1709bdf..c9fcc7a 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
@@ -39,8 +39,7 @@ private[spark] object RpcEnv {
     val rpcEnvNames = Map("akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnvFactory")
     val rpcEnvName = conf.get("spark.rpc", "akka")
     val rpcEnvFactoryClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName)
-    Class.forName(rpcEnvFactoryClassName, true, Utils.getContextOrSparkClassLoader).
-      newInstance().asInstanceOf[RpcEnvFactory]
+    Utils.classForName(rpcEnvFactoryClassName).newInstance().asInstanceOf[RpcEnvFactory]
   }
 
   def create(

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index 698d138..4a5274b 100644
--- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
@@ -62,8 +62,11 @@ private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoa
   extends DeserializationStream {
 
   private val objIn = new ObjectInputStream(in) {
-    override def resolveClass(desc: ObjectStreamClass): Class[_] =
+    override def resolveClass(desc: ObjectStreamClass): Class[_] = {
+      // scalastyle:off classforname
       Class.forName(desc.getName, false, loader)
+      // scalastyle:on classforname
+    }
   }
 
   def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T]

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index ed35cff..7cb6e08 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -102,6 +102,7 @@ class KryoSerializer(conf: SparkConf)
     kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer())
 
     try {
+      // scalastyle:off classforname
       // Use the default classloader when calling the user registrator.
       Thread.currentThread.setContextClassLoader(classLoader)
       // Register classes given through spark.kryo.classesToRegister.
@@ -111,6 +112,7 @@ class KryoSerializer(conf: SparkConf)
       userRegistrator
         .map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator])
         .foreach { reg => reg.registerClasses(kryo) }
+      // scalastyle:on classforname
     } catch {
       case e: Exception =>
         throw new SparkException(s"Failed to register classes with Kryo", e)

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala
index cc2f050..a1b1e16 100644
--- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala
@@ -407,7 +407,9 @@ private[spark] object SerializationDebugger extends Logging {
 
     /** ObjectStreamClass$ClassDataSlot.desc field */
     val DescField: Field = {
+      // scalastyle:off classforname
       val f = Class.forName("java.io.ObjectStreamClass$ClassDataSlot").getDeclaredField("desc")
+      // scalastyle:on classforname
       f.setAccessible(true)
       f
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala
index 291394e..db965d5 100644
--- a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala
@@ -192,7 +192,7 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId:
       .getOrElse(ExternalBlockStore.DEFAULT_BLOCK_MANAGER_NAME)
 
     try {
-      val instance = Class.forName(clsName)
+      val instance = Utils.classForName(clsName)
         .newInstance()
         .asInstanceOf[ExternalBlockManager]
       instance.init(blockManager, executorId)

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
index 305de4c..43626b4 100644
--- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
@@ -448,10 +448,12 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM
         if (op == INVOKESPECIAL && name == "<init>" && argTypes.length > 0
             && argTypes(0).toString.startsWith("L") // is it an object?
             && argTypes(0).getInternalName == myName) {
+          // scalastyle:off classforname
           output += Class.forName(
               owner.replace('/', '.'),
               false,
               Thread.currentThread.getContextClassLoader)
+          // scalastyle:on classforname
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
index 0180399..7d84468 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -124,9 +124,11 @@ object SizeEstimator extends Logging {
       val server = ManagementFactory.getPlatformMBeanServer()
 
       // NOTE: This should throw an exception in non-Sun JVMs
+      // scalastyle:off classforname
       val hotSpotMBeanClass = Class.forName("com.sun.management.HotSpotDiagnosticMXBean")
       val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption",
           Class.forName("java.lang.String"))
+      // scalastyle:on classforname
 
       val bean = ManagementFactory.newPlatformMXBeanProxy(server,
         hotSpotMBeanName, hotSpotMBeanClass)

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index b6b9321..e6374f1 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -113,8 +113,11 @@ private[spark] object Utils extends Logging {
   def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = {
     val bis = new ByteArrayInputStream(bytes)
     val ois = new ObjectInputStream(bis) {
-      override def resolveClass(desc: ObjectStreamClass): Class[_] =
+      override def resolveClass(desc: ObjectStreamClass): Class[_] = {
+        // scalastyle:off classforname
         Class.forName(desc.getName, false, loader)
+        // scalastyle:on classforname
+      }
     }
     ois.readObject.asInstanceOf[T]
   }
@@ -177,12 +180,16 @@ private[spark] object Utils extends Logging {
 
   /** Determines whether the provided class is loadable in the current thread. */
   def classIsLoadable(clazz: String): Boolean = {
+    // scalastyle:off classforname
     Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess
+    // scalastyle:on classforname
   }
 
+  // scalastyle:off classforname
   /** Preferred alternative to Class.forName(className) */
   def classForName(className: String): Class[_] = {
     Class.forName(className, true, getContextOrSparkClassLoader)
+    // scalastyle:on classforname
   }
 
   /**
@@ -2266,7 +2273,7 @@ private [util] class SparkShutdownHookManager {
     val hookTask = new Runnable() {
       override def run(): Unit = runAll()
     }
-    Try(Class.forName("org.apache.hadoop.util.ShutdownHookManager")) match {
+    Try(Utils.classForName("org.apache.hadoop.util.ShutdownHookManager")) match {
       case Success(shmClass) =>
         val fsPriority = classOf[FileSystem].getField("SHUTDOWN_HOOK_PRIORITY").get()
           .asInstanceOf[Int]

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/test/scala/org/apache/spark/FileSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index 1d8fade..418763f 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -179,6 +179,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
   }
 
   test("object files of classes from a JAR") {
+    // scalastyle:off classforname
     val original = Thread.currentThread().getContextClassLoader
     val className = "FileSuiteObjectFileTest"
     val jar = TestUtils.createJarWithClasses(Seq(className))
@@ -201,6 +202,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
     finally {
       Thread.currentThread().setContextClassLoader(original)
     }
+    // scalastyle:on classforname
   }
 
   test("write SequenceFile using new Hadoop API") {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
index f89e3d0..dba46f1 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark
 
 import org.scalatest.PrivateMethodTester
 
+import org.apache.spark.util.Utils
 import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
 import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend}
 import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
@@ -131,7 +132,7 @@ class SparkContextSchedulerCreationSuite
   def testYarn(master: String, expectedClassName: String) {
     try {
       val sched = createTaskScheduler(master)
-      assert(sched.getClass === Class.forName(expectedClassName))
+      assert(sched.getClass === Utils.classForName(expectedClassName))
     } catch {
       case e: SparkException =>
         assert(e.getMessage.contains("YARN mode not available"))

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index e7878bd..343d28e 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -541,8 +541,8 @@ object JarCreationTest extends Logging {
     val result = sc.makeRDD(1 to 100, 10).mapPartitions { x =>
       var exception: String = null
       try {
-        Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
-        Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
+        Utils.classForName(args(0))
+        Utils.classForName(args(1))
       } catch {
         case t: Throwable =>
           exception = t + "\n" + t.getStackTraceString

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
index 08215a2..05013fb 100644
--- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
@@ -22,11 +22,12 @@ import java.sql._
 import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite}
+import org.apache.spark.util.Utils
 
 class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext {
 
   before {
-    Class.forName("org.apache.derby.jdbc.EmbeddedDriver")
+    Utils.classForName("org.apache.derby.jdbc.EmbeddedDriver")
     val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true")
     try {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala
index 63a8480..353b974 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala
@@ -59,7 +59,9 @@ object KryoDistributedTest {
   class AppJarRegistrator extends KryoRegistrator {
     override def registerClasses(k: Kryo) {
       val classLoader = Thread.currentThread.getContextClassLoader
+      // scalastyle:off classforname
       k.register(Class.forName(AppJarRegistrator.customClassName, true, classLoader))
+      // scalastyle:on classforname
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala
index 4212554..d3d464e 100644
--- a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala
@@ -84,7 +84,9 @@ class MutableURLClassLoaderSuite extends SparkFunSuite {
     try {
       sc.makeRDD(1 to 5, 2).mapPartitions { x =>
         val loader = Thread.currentThread().getContextClassLoader
+        // scalastyle:off classforname
         Class.forName(className, true, loader).newInstance()
+        // scalastyle:on classforname
         Seq().iterator
       }.count()
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala
----------------------------------------------------------------------
diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala
index 17cbc67..d87b869 100644
--- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala
+++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala
@@ -113,7 +113,9 @@ private[sink] object Logging {
   try {
     // We use reflection here to handle the case where users remove the
     // slf4j-to-jul bridge order to route their logs to JUL.
+    // scalastyle:off classforname
     val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler")
+    // scalastyle:on classforname
     bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null)
     val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean]
     if (!installed) {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
index 5c07b41..74a7de1 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
@@ -121,7 +121,7 @@ private[graphx] object BytecodeUtils {
           override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) {
             if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) {
               if (!skipClass(owner)) {
-                methodsInvoked.add((Class.forName(owner.replace("/", ".")), name))
+                methodsInvoked.add((Utils.classForName(owner.replace("/", ".")), name))
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
----------------------------------------------------------------------
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index 8791618..4ee605f 100644
--- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -1079,8 +1079,10 @@ import org.apache.spark.annotation.DeveloperApi
       throw new EvalException("Failed to load '" + path + "': " + ex.getMessage, ex)
 
     private def load(path: String): Class[_] = {
+      // scalastyle:off classforname
       try Class.forName(path, true, classLoader)
       catch { case ex: Throwable => evalError(path, unwrap(ex)) }
+      // scalastyle:on classforname
     }
 
     lazy val evalClass = load(evalPath)

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/scalastyle-config.xml
----------------------------------------------------------------------
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index 4961170..b5e2e88 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -150,6 +150,17 @@ This file is divided into 3 sections:
       // scalastyle:on println]]></customMessage>
   </check>
 
+  <check customId="classforname" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
+    <parameters><parameter name="regex">Class\.forName</parameter></parameters>
+    <customMessage><![CDATA[
+      Are you sure that you want to use Class.forName? In most cases, you should use Utils.classForName instead.
+      If you must use Class.forName, wrap the code block with
+      // scalastyle:off classforname
+      Class.forName(...)
+      // scalastyle:on classforname
+    ]]></customMessage>
+  </check>
+
   <!-- ================================================================================ -->
   <!--       rules we'd like to enforce, but haven't cleaned up the codebase yet        -->
   <!-- ================================================================================ -->

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
index 5771822..da83a7f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
@@ -27,6 +27,7 @@ import org.json4s.jackson.JsonMethods._
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.util.Utils
 
 
 /**
@@ -146,7 +147,7 @@ object DataType {
     ("pyClass", _),
     ("sqlType", _),
     ("type", JString("udt"))) =>
-      Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]]
+      Utils.classForName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]]
   }
 
   private def parseStructField(json: JValue): StructField = json match {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 477dea9..46bd60d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -554,8 +554,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
     val className = beanClass.getName
     val rowRdd = rdd.mapPartitions { iter =>
       // BeanInfo is not serializable so we must rediscover it remotely for each partition.
-      val localBeanInfo = Introspector.getBeanInfo(
-        Class.forName(className, true, Utils.getContextOrSparkClassLoader))
+      val localBeanInfo = Introspector.getBeanInfo(Utils.classForName(className))
       val extractors =
         localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod)
       val methodsToConverts = extractors.zip(attributeSeq).map { case (e, attr) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index 704cf56..e0bea65 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
 import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.{DataFrame, SQLContext}
+import org.apache.spark.util.Utils
 
 /**
  * Relation that consists of data stored in a Parquet columnar format.
@@ -108,7 +109,7 @@ private[sql] object ParquetRelation {
     //
     // Therefore we need to force the class to be loaded.
     // This should really be resolved by Parquet.
-    Class.forName(classOf[ParquetLog].getName)
+    Utils.classForName(classOf[ParquetLog].getName)
 
     // Note: Logger.getLogger("parquet") has a default logger
     // that appends to Console which needs to be cleared.
@@ -119,12 +120,12 @@ private[sql] object ParquetRelation {
     // Disables a WARN log message in ParquetOutputCommitter.  We first ensure that
     // ParquetOutputCommitter is loaded and the static LOG field gets initialized.
     // See https://issues.apache.org/jira/browse/SPARK-5968 for details
-    Class.forName(classOf[ParquetOutputCommitter].getName)
+    Utils.classForName(classOf[ParquetOutputCommitter].getName)
     JLogger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF)
 
     // Similar as above, disables a unnecessary WARN log message in ParquetRecordReader.
     // See https://issues.apache.org/jira/browse/PARQUET-220 for details
-    Class.forName(classOf[ParquetRecordReader[_]].getName)
+    Utils.classForName(classOf[ParquetRecordReader[_]].getName)
     JLogger.getLogger(classOf[ParquetRecordReader[_]].getName).setLevel(Level.OFF)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 566a52d..0f82f13 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -26,6 +26,7 @@ import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
 
 class JDBCSuite extends SparkFunSuite with BeforeAndAfter {
   val url = "jdbc:h2:mem:testdb0"
@@ -46,7 +47,7 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter {
   import ctx.sql
 
   before {
-    Class.forName("org.h2.Driver")
+    Utils.classForName("org.h2.Driver")
     // Extra properties that will be specified for our database. We need these to test
     // usage of parameters from OPTIONS clause in queries.
     val properties = new Properties()

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
index d949ef4..84b52ca 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
@@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfter
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.{SaveMode, Row}
 import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
 
 class JDBCWriteSuite extends SparkFunSuite with BeforeAndAfter {
   val url = "jdbc:h2:mem:testdb2"
@@ -41,7 +42,7 @@ class JDBCWriteSuite extends SparkFunSuite with BeforeAndAfter {
   import ctx.sql
 
   before {
-    Class.forName("org.h2.Driver")
+    Utils.classForName("org.h2.Driver")
     conn = DriverManager.getConnection(url)
     conn.prepareStatement("create schema test").executeUpdate()
 

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 301aa5a..39b3152 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -417,7 +417,7 @@ object ServerMode extends Enumeration {
 }
 
 abstract class HiveThriftJdbcTest extends HiveThriftServer2Test {
-  Class.forName(classOf[HiveDriver].getCanonicalName)
+  Utils.classForName(classOf[HiveDriver].getCanonicalName)
 
   private def jdbcUri = if (mode == ServerMode.http) {
     s"""jdbc:hive2://localhost:$serverPort/

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index d65d29d..dc35569 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -78,9 +78,7 @@ class HadoopTableReader(
   override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] =
     makeRDDForTable(
       hiveTable,
-      Class.forName(
-        relation.tableDesc.getSerdeClassName, true, Utils.getContextOrSparkClassLoader)
-        .asInstanceOf[Class[Deserializer]],
+      Utils.classForName(relation.tableDesc.getSerdeClassName).asInstanceOf[Class[Deserializer]],
       filterOpt = None)
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index 1f280c6..8adda54 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -21,9 +21,6 @@ import java.io.{File, PrintStream}
 import java.util.{Map => JMap}
 import javax.annotation.concurrent.GuardedBy
 
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.util.CircularBuffer
-
 import scala.collection.JavaConversions._
 import scala.language.reflectiveCalls
 
@@ -37,7 +34,9 @@ import org.apache.hadoop.hive.ql.session.SessionState
 import org.apache.hadoop.hive.ql.{Driver, metadata}
 
 import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.execution.QueryExecutionException
+import org.apache.spark.util.{CircularBuffer, Utils}
 
 
 /**
@@ -249,10 +248,10 @@ private[hive] class ClientWrapper(
   }
 
   private def toInputFormat(name: String) =
-    Class.forName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]]
+    Utils.classForName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]]
 
   private def toOutputFormat(name: String) =
-    Class.forName(name)
+    Utils.classForName(name)
       .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]]
 
   private def toQlTable(table: HiveTable): metadata.Table = {

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
index 917900e..bee2ecb 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -120,8 +120,8 @@ object SparkSubmitClassLoaderTest extends Logging {
     logInfo("Testing load classes at the driver side.")
     // First, we load classes at driver side.
     try {
-      Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
-      Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
+      Utils.classForName(args(0))
+      Utils.classForName(args(1))
     } catch {
       case t: Throwable =>
         throw new Exception("Could not load user class from jar:\n", t)
@@ -131,8 +131,8 @@ object SparkSubmitClassLoaderTest extends Logging {
     val result = df.mapPartitions { x =>
       var exception: String = null
       try {
-        Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
-        Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
+        Utils.classForName(args(0))
+        Utils.classForName(args(1))
       } catch {
         case t: Throwable =>
           exception = t + "\n" + t.getStackTraceString

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index f5d4185..9f2117a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -22,7 +22,7 @@ import scala.util.{Failure, Success, Try}
 import org.apache.spark.{SparkEnv, Logging}
 import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time}
 import org.apache.spark.streaming.util.RecurringTimer
-import org.apache.spark.util.{Clock, EventLoop, ManualClock}
+import org.apache.spark.util.{Utils, Clock, EventLoop, ManualClock}
 
 /** Event classes for JobGenerator */
 private[scheduler] sealed trait JobGeneratorEvent
@@ -47,11 +47,11 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
     val clockClass = ssc.sc.conf.get(
       "spark.streaming.clock", "org.apache.spark.util.SystemClock")
     try {
-      Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+      Utils.classForName(clockClass).newInstance().asInstanceOf[Clock]
     } catch {
       case e: ClassNotFoundException if clockClass.startsWith("org.apache.spark.streaming") =>
         val newClockClass = clockClass.replace("org.apache.spark.streaming", "org.apache.spark")
-        Class.forName(newClockClass).newInstance().asInstanceOf[Clock]
+        Utils.classForName(newClockClass).newInstance().asInstanceOf[Clock]
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
----------------------------------------------------------------------
diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
index 9483d2b..9418beb 100644
--- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off classforname
 package org.apache.spark.tools
 
 import java.io.File
@@ -188,3 +189,4 @@ object GenerateMIMAIgnore {
     classes
   }
 }
+// scalastyle:on classforname

http://git-wip-us.apache.org/repos/asf/spark/blob/11e5c372/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index f0af6f8..f86b6d1 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -732,9 +732,9 @@ private[spark] class Client(
       }
     val amClass =
       if (isClusterMode) {
-        Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName
+        Utils.classForName("org.apache.spark.deploy.yarn.ApplicationMaster").getName
       } else {
-        Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName
+        Utils.classForName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName
       }
     if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
       args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org