Skip to content

Commit 11e5c37

Browse files
JoshRosenrxin
authored andcommitted
[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 <[email protected]> Closes apache#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
1 parent 740b034 commit 11e5c37

File tree

49 files changed

+117
-84
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

49 files changed

+117
-84
lines changed

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

+1-1
Original file line numberDiff line numberDiff line change
@@ -159,7 +159,7 @@ private object Logging {
159159
try {
160160
// We use reflection here to handle the case where users remove the
161161
// slf4j-to-jul bridge order to route their logs to JUL.
162-
val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler")
162+
val bridgeClass = Utils.classForName("org.slf4j.bridge.SLF4JBridgeHandler")
163163
bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null)
164164
val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean]
165165
if (!installed) {

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

+5-6
Original file line numberDiff line numberDiff line change
@@ -1968,7 +1968,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
19681968
for (className <- listenerClassNames) {
19691969
// Use reflection to find the right constructor
19701970
val constructors = {
1971-
val listenerClass = Class.forName(className)
1971+
val listenerClass = Utils.classForName(className)
19721972
listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]]
19731973
}
19741974
val constructorTakingSparkConf = constructors.find { c =>
@@ -2503,7 +2503,7 @@ object SparkContext extends Logging {
25032503
"\"yarn-standalone\" is deprecated as of Spark 1.0. Use \"yarn-cluster\" instead.")
25042504
}
25052505
val scheduler = try {
2506-
val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
2506+
val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
25072507
val cons = clazz.getConstructor(classOf[SparkContext])
25082508
cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
25092509
} catch {
@@ -2515,7 +2515,7 @@ object SparkContext extends Logging {
25152515
}
25162516
val backend = try {
25172517
val clazz =
2518-
Class.forName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend")
2518+
Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend")
25192519
val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
25202520
cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
25212521
} catch {
@@ -2528,8 +2528,7 @@ object SparkContext extends Logging {
25282528

25292529
case "yarn-client" =>
25302530
val scheduler = try {
2531-
val clazz =
2532-
Class.forName("org.apache.spark.scheduler.cluster.YarnScheduler")
2531+
val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnScheduler")
25332532
val cons = clazz.getConstructor(classOf[SparkContext])
25342533
cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
25352534

@@ -2541,7 +2540,7 @@ object SparkContext extends Logging {
25412540

25422541
val backend = try {
25432542
val clazz =
2544-
Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend")
2543+
Utils.classForName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend")
25452544
val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
25462545
cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
25472546
} catch {

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

+1-1
Original file line numberDiff line numberDiff line change
@@ -261,7 +261,7 @@ object SparkEnv extends Logging {
261261

262262
// Create an instance of the class with the given name, possibly initializing it with our conf
263263
def instantiateClass[T](className: String): T = {
264-
val cls = Class.forName(className, true, Utils.getContextOrSparkClassLoader)
264+
val cls = Utils.classForName(className)
265265
// Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just
266266
// SparkConf, then one taking no arguments
267267
try {

core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala

+2-16
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler}
2626

2727
import org.apache.spark.Logging
2828
import org.apache.spark.api.r.SerDe._
29+
import org.apache.spark.util.Utils
2930

3031
/**
3132
* Handler for RBackend
@@ -88,21 +89,6 @@ private[r] class RBackendHandler(server: RBackend)
8889
ctx.close()
8990
}
9091

91-
// Looks up a class given a class name. This function first checks the
92-
// current class loader and if a class is not found, it looks up the class
93-
// in the context class loader. Address [SPARK-5185]
94-
def getStaticClass(objId: String): Class[_] = {
95-
try {
96-
val clsCurrent = Class.forName(objId)
97-
clsCurrent
98-
} catch {
99-
// Use contextLoader if we can't find the JAR in the system class loader
100-
case e: ClassNotFoundException =>
101-
val clsContext = Class.forName(objId, true, Thread.currentThread().getContextClassLoader)
102-
clsContext
103-
}
104-
}
105-
10692
def handleMethodCall(
10793
isStatic: Boolean,
10894
objId: String,
@@ -113,7 +99,7 @@ private[r] class RBackendHandler(server: RBackend)
11399
var obj: Object = null
114100
try {
115101
val cls = if (isStatic) {
116-
getStaticClass(objId)
102+
Utils.classForName(objId)
117103
} else {
118104
JVMObjectTracker.get(objId) match {
119105
case None => throw new IllegalArgumentException("Object not found " + objId)

core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala

+2-1
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicLong
2222
import scala.reflect.ClassTag
2323

2424
import org.apache.spark._
25+
import org.apache.spark.util.Utils
2526

2627
private[spark] class BroadcastManager(
2728
val isDriver: Boolean,
@@ -42,7 +43,7 @@ private[spark] class BroadcastManager(
4243
conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory")
4344

4445
broadcastFactory =
45-
Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
46+
Utils.classForName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
4647

4748
// Initialize appropriate BroadcastFactory and BroadcastObject
4849
broadcastFactory.initialize(isDriver, conf, securityManager)

core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -178,7 +178,7 @@ class SparkHadoopUtil extends Logging {
178178

179179
private def getFileSystemThreadStatisticsMethod(methodName: String): Method = {
180180
val statisticsDataClass =
181-
Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData")
181+
Utils.classForName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData")
182182
statisticsDataClass.getDeclaredMethod(methodName)
183183
}
184184

@@ -356,7 +356,7 @@ object SparkHadoopUtil {
356356
System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
357357
if (yarnMode) {
358358
try {
359-
Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
359+
Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
360360
.newInstance()
361361
.asInstanceOf[SparkHadoopUtil]
362362
} catch {

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -624,7 +624,7 @@ object SparkSubmit {
624624
var mainClass: Class[_] = null
625625

626626
try {
627-
mainClass = Class.forName(childMainClass, true, loader)
627+
mainClass = Utils.classForName(childMainClass)
628628
} catch {
629629
case e: ClassNotFoundException =>
630630
e.printStackTrace(printStream)

core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -576,7 +576,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
576576
System.setSecurityManager(sm)
577577

578578
try {
579-
Class.forName(mainClass).getMethod("main", classOf[Array[String]])
579+
Utils.classForName(mainClass).getMethod("main", classOf[Array[String]])
580580
.invoke(null, Array(HELP))
581581
} catch {
582582
case e: InvocationTargetException =>

core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -228,7 +228,7 @@ object HistoryServer extends Logging {
228228

229229
val providerName = conf.getOption("spark.history.provider")
230230
.getOrElse(classOf[FsHistoryProvider].getName())
231-
val provider = Class.forName(providerName)
231+
val provider = Utils.classForName(providerName)
232232
.getConstructor(classOf[SparkConf])
233233
.newInstance(conf)
234234
.asInstanceOf[ApplicationHistoryProvider]

core/src/main/scala/org/apache/spark/deploy/master/Master.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -172,7 +172,7 @@ private[master] class Master(
172172
new FileSystemRecoveryModeFactory(conf, SerializationExtension(actorSystem))
173173
(fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this))
174174
case "CUSTOM" =>
175-
val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory"))
175+
val clazz = Utils.classForName(conf.get("spark.deploy.recoveryMode.factory"))
176176
val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serialization])
177177
.newInstance(conf, SerializationExtension(actorSystem))
178178
.asInstanceOf[StandaloneRecoveryModeFactory]

core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ private[spark] object SubmitRestProtocolMessage {
128128
*/
129129
def fromJson(json: String): SubmitRestProtocolMessage = {
130130
val className = parseAction(json)
131-
val clazz = Class.forName(packagePrefix + "." + className)
131+
val clazz = Utils.classForName(packagePrefix + "." + className)
132132
.asSubclass[SubmitRestProtocolMessage](classOf[SubmitRestProtocolMessage])
133133
fromJson(json, clazz)
134134
}

core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ object DriverWrapper {
5353
Thread.currentThread.setContextClassLoader(loader)
5454

5555
// Delegate to supplied main class
56-
val clazz = Class.forName(mainClass, true, loader)
56+
val clazz = Utils.classForName(mainClass)
5757
val mainMethod = clazz.getMethod("main", classOf[Array[String]])
5858
mainMethod.invoke(null, extraArgs.toArray[String])
5959

core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala

+2
Original file line numberDiff line numberDiff line change
@@ -149,6 +149,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
149149
val ibmVendor = System.getProperty("java.vendor").contains("IBM")
150150
var totalMb = 0
151151
try {
152+
// scalastyle:off classforname
152153
val bean = ManagementFactory.getOperatingSystemMXBean()
153154
if (ibmVendor) {
154155
val beanClass = Class.forName("com.ibm.lang.management.OperatingSystemMXBean")
@@ -159,6 +160,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
159160
val method = beanClass.getDeclaredMethod("getTotalPhysicalMemorySize")
160161
totalMb = (method.invoke(bean).asInstanceOf[Long] / 1024 / 1024).toInt
161162
}
163+
// scalastyle:on classforname
162164
} catch {
163165
case e: Exception => {
164166
totalMb = 2*1024

core/src/main/scala/org/apache/spark/executor/Executor.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -356,7 +356,7 @@ private[spark] class Executor(
356356
logInfo("Using REPL class URI: " + classUri)
357357
try {
358358
val _userClassPathFirst: java.lang.Boolean = userClassPathFirst
359-
val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader")
359+
val klass = Utils.classForName("org.apache.spark.repl.ExecutorClassLoader")
360360
.asInstanceOf[Class[_ <: ClassLoader]]
361361
val constructor = klass.getConstructor(classOf[SparkConf], classOf[String],
362362
classOf[ClassLoader], classOf[Boolean])

core/src/main/scala/org/apache/spark/io/CompressionCodec.scala

+1-2
Original file line numberDiff line numberDiff line change
@@ -63,8 +63,7 @@ private[spark] object CompressionCodec {
6363
def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
6464
val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName)
6565
val codec = try {
66-
val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader)
67-
.getConstructor(classOf[SparkConf])
66+
val ctor = Utils.classForName(codecClass).getConstructor(classOf[SparkConf])
6867
Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec])
6968
} catch {
7069
case e: ClassNotFoundException => None

core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala

+3-2
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter}
2626

2727
import org.apache.spark.executor.CommitDeniedException
2828
import org.apache.spark.{Logging, SparkEnv, TaskContext}
29+
import org.apache.spark.util.{Utils => SparkUtils}
2930

3031
private[spark]
3132
trait SparkHadoopMapRedUtil {
@@ -64,10 +65,10 @@ trait SparkHadoopMapRedUtil {
6465

6566
private def firstAvailableClass(first: String, second: String): Class[_] = {
6667
try {
67-
Class.forName(first)
68+
SparkUtils.classForName(first)
6869
} catch {
6970
case e: ClassNotFoundException =>
70-
Class.forName(second)
71+
SparkUtils.classForName(second)
7172
}
7273
}
7374
}

core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala

+5-4
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import java.lang.{Boolean => JBoolean, Integer => JInteger}
2121

2222
import org.apache.hadoop.conf.Configuration
2323
import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID}
24+
import org.apache.spark.util.Utils
2425

2526
private[spark]
2627
trait SparkHadoopMapReduceUtil {
@@ -46,7 +47,7 @@ trait SparkHadoopMapReduceUtil {
4647
isMap: Boolean,
4748
taskId: Int,
4849
attemptId: Int): TaskAttemptID = {
49-
val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID")
50+
val klass = Utils.classForName("org.apache.hadoop.mapreduce.TaskAttemptID")
5051
try {
5152
// First, attempt to use the old-style constructor that takes a boolean isMap
5253
// (not available in YARN)
@@ -57,7 +58,7 @@ trait SparkHadoopMapReduceUtil {
5758
} catch {
5859
case exc: NoSuchMethodException => {
5960
// If that failed, look for the new constructor that takes a TaskType (not available in 1.x)
60-
val taskTypeClass = Class.forName("org.apache.hadoop.mapreduce.TaskType")
61+
val taskTypeClass = Utils.classForName("org.apache.hadoop.mapreduce.TaskType")
6162
.asInstanceOf[Class[Enum[_]]]
6263
val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke(
6364
taskTypeClass, if (isMap) "MAP" else "REDUCE")
@@ -71,10 +72,10 @@ trait SparkHadoopMapReduceUtil {
7172

7273
private def firstAvailableClass(first: String, second: String): Class[_] = {
7374
try {
74-
Class.forName(first)
75+
Utils.classForName(first)
7576
} catch {
7677
case e: ClassNotFoundException =>
77-
Class.forName(second)
78+
Utils.classForName(second)
7879
}
7980
}
8081
}

core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala

+4-2
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@ package org.apache.spark.metrics
2020
import java.util.Properties
2121
import java.util.concurrent.TimeUnit
2222

23+
import org.apache.spark.util.Utils
24+
2325
import scala.collection.mutable
2426

2527
import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry}
@@ -166,7 +168,7 @@ private[spark] class MetricsSystem private (
166168
sourceConfigs.foreach { kv =>
167169
val classPath = kv._2.getProperty("class")
168170
try {
169-
val source = Class.forName(classPath).newInstance()
171+
val source = Utils.classForName(classPath).newInstance()
170172
registerSource(source.asInstanceOf[Source])
171173
} catch {
172174
case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e)
@@ -182,7 +184,7 @@ private[spark] class MetricsSystem private (
182184
val classPath = kv._2.getProperty("class")
183185
if (null != classPath) {
184186
try {
185-
val sink = Class.forName(classPath)
187+
val sink = Utils.classForName(classPath)
186188
.getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager])
187189
.newInstance(kv._2, registry, securityMgr)
188190
if (kv._1 == "servlet") {

core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala

+3-3
Original file line numberDiff line numberDiff line change
@@ -383,11 +383,11 @@ private[spark] object HadoopRDD extends Logging {
383383

384384
private[spark] class SplitInfoReflections {
385385
val inputSplitWithLocationInfo =
386-
Class.forName("org.apache.hadoop.mapred.InputSplitWithLocationInfo")
386+
Utils.classForName("org.apache.hadoop.mapred.InputSplitWithLocationInfo")
387387
val getLocationInfo = inputSplitWithLocationInfo.getMethod("getLocationInfo")
388-
val newInputSplit = Class.forName("org.apache.hadoop.mapreduce.InputSplit")
388+
val newInputSplit = Utils.classForName("org.apache.hadoop.mapreduce.InputSplit")
389389
val newGetLocationInfo = newInputSplit.getMethod("getLocationInfo")
390-
val splitLocationInfo = Class.forName("org.apache.hadoop.mapred.SplitLocationInfo")
390+
val splitLocationInfo = Utils.classForName("org.apache.hadoop.mapred.SplitLocationInfo")
391391
val isInMemory = splitLocationInfo.getMethod("isInMemory")
392392
val getLocation = splitLocationInfo.getMethod("getLocation")
393393
}

core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala

+1-2
Original file line numberDiff line numberDiff line change
@@ -39,8 +39,7 @@ private[spark] object RpcEnv {
3939
val rpcEnvNames = Map("akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnvFactory")
4040
val rpcEnvName = conf.get("spark.rpc", "akka")
4141
val rpcEnvFactoryClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName)
42-
Class.forName(rpcEnvFactoryClassName, true, Utils.getContextOrSparkClassLoader).
43-
newInstance().asInstanceOf[RpcEnvFactory]
42+
Utils.classForName(rpcEnvFactoryClassName).newInstance().asInstanceOf[RpcEnvFactory]
4443
}
4544

4645
def create(

core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala

+4-1
Original file line numberDiff line numberDiff line change
@@ -62,8 +62,11 @@ private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoa
6262
extends DeserializationStream {
6363

6464
private val objIn = new ObjectInputStream(in) {
65-
override def resolveClass(desc: ObjectStreamClass): Class[_] =
65+
override def resolveClass(desc: ObjectStreamClass): Class[_] = {
66+
// scalastyle:off classforname
6667
Class.forName(desc.getName, false, loader)
68+
// scalastyle:on classforname
69+
}
6770
}
6871

6972
def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T]

core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala

+2
Original file line numberDiff line numberDiff line change
@@ -102,6 +102,7 @@ class KryoSerializer(conf: SparkConf)
102102
kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer())
103103

104104
try {
105+
// scalastyle:off classforname
105106
// Use the default classloader when calling the user registrator.
106107
Thread.currentThread.setContextClassLoader(classLoader)
107108
// Register classes given through spark.kryo.classesToRegister.
@@ -111,6 +112,7 @@ class KryoSerializer(conf: SparkConf)
111112
userRegistrator
112113
.map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator])
113114
.foreach { reg => reg.registerClasses(kryo) }
115+
// scalastyle:on classforname
114116
} catch {
115117
case e: Exception =>
116118
throw new SparkException(s"Failed to register classes with Kryo", e)

core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala

+2
Original file line numberDiff line numberDiff line change
@@ -407,7 +407,9 @@ private[spark] object SerializationDebugger extends Logging {
407407

408408
/** ObjectStreamClass$ClassDataSlot.desc field */
409409
val DescField: Field = {
410+
// scalastyle:off classforname
410411
val f = Class.forName("java.io.ObjectStreamClass$ClassDataSlot").getDeclaredField("desc")
412+
// scalastyle:on classforname
411413
f.setAccessible(true)
412414
f
413415
}

core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -192,7 +192,7 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId:
192192
.getOrElse(ExternalBlockStore.DEFAULT_BLOCK_MANAGER_NAME)
193193

194194
try {
195-
val instance = Class.forName(clsName)
195+
val instance = Utils.classForName(clsName)
196196
.newInstance()
197197
.asInstanceOf[ExternalBlockManager]
198198
instance.init(blockManager, executorId)

core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala

+2
Original file line numberDiff line numberDiff line change
@@ -448,10 +448,12 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM
448448
if (op == INVOKESPECIAL && name == "<init>" && argTypes.length > 0
449449
&& argTypes(0).toString.startsWith("L") // is it an object?
450450
&& argTypes(0).getInternalName == myName) {
451+
// scalastyle:off classforname
451452
output += Class.forName(
452453
owner.replace('/', '.'),
453454
false,
454455
Thread.currentThread.getContextClassLoader)
456+
// scalastyle:on classforname
455457
}
456458
}
457459
}

0 commit comments

Comments
 (0)