Skip to content

Commit 2d085c1

Browse files
committed
[SPARK-25984][CORE][SQL][STREAMING] Remove deprecated .newInstance(), primitive box class constructor calls
## What changes were proposed in this pull request? Deprecated in Java 11, replace Class.newInstance with Class.getConstructor.getInstance, and primtive wrapper class constructors with valueOf or equivalent ## How was this patch tested? Existing tests. Closes #22988 from srowen/SPARK-25984. Authored-by: Sean Owen <[email protected]> Signed-off-by: Sean Owen <[email protected]>
1 parent d66a4e8 commit 2d085c1

File tree

70 files changed

+190
-174
lines changed

Some content is hidden

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

70 files changed

+190
-174
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -261,7 +261,7 @@ object SparkEnv extends Logging {
261261
// SparkConf, then one taking no arguments
262262
try {
263263
cls.getConstructor(classOf[SparkConf], java.lang.Boolean.TYPE)
264-
.newInstance(conf, new java.lang.Boolean(isDriver))
264+
.newInstance(conf, java.lang.Boolean.valueOf(isDriver))
265265
.asInstanceOf[T]
266266
} catch {
267267
case _: NoSuchMethodException =>

core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,8 @@ private[python] object Converter extends Logging {
4343
defaultConverter: Converter[Any, Any]): Converter[Any, Any] = {
4444
converterClass.map { cc =>
4545
Try {
46-
val c = Utils.classForName(cc).newInstance().asInstanceOf[Converter[Any, Any]]
46+
val c = Utils.classForName(cc).getConstructor().
47+
newInstance().asInstanceOf[Converter[Any, Any]]
4748
logInfo(s"Loaded converter: $cc")
4849
c
4950
} match {

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -74,9 +74,9 @@ private[spark] object SerDe {
7474
jvmObjectTracker: JVMObjectTracker): Object = {
7575
dataType match {
7676
case 'n' => null
77-
case 'i' => new java.lang.Integer(readInt(dis))
78-
case 'd' => new java.lang.Double(readDouble(dis))
79-
case 'b' => new java.lang.Boolean(readBoolean(dis))
77+
case 'i' => java.lang.Integer.valueOf(readInt(dis))
78+
case 'd' => java.lang.Double.valueOf(readDouble(dis))
79+
case 'b' => java.lang.Boolean.valueOf(readBoolean(dis))
8080
case 'c' => readString(dis)
8181
case 'e' => readMap(dis, jvmObjectTracker)
8282
case 'r' => readBytes(dis)

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -829,7 +829,7 @@ private[spark] class SparkSubmit extends Logging {
829829
}
830830

831831
val app: SparkApplication = if (classOf[SparkApplication].isAssignableFrom(mainClass)) {
832-
mainClass.newInstance().asInstanceOf[SparkApplication]
832+
mainClass.getConstructor().newInstance().asInstanceOf[SparkApplication]
833833
} else {
834834
// SPARK-4170
835835
if (classOf[scala.App].isAssignableFrom(mainClass)) {

core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -91,7 +91,7 @@ class HadoopMapReduceCommitProtocol(
9191
private def stagingDir = new Path(path, ".spark-staging-" + jobId)
9292

9393
protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = {
94-
val format = context.getOutputFormatClass.newInstance()
94+
val format = context.getOutputFormatClass.getConstructor().newInstance()
9595
// If OutputFormat is Configurable, we should set conf to it.
9696
format match {
9797
case c: Configurable => c.setConf(context.getConfiguration)

core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -256,7 +256,7 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf)
256256
private def getOutputFormat(): OutputFormat[K, V] = {
257257
require(outputFormat != null, "Must call initOutputFormat first.")
258258

259-
outputFormat.newInstance()
259+
outputFormat.getConstructor().newInstance()
260260
}
261261

262262
// --------------------------------------------------------------------------
@@ -379,7 +379,7 @@ class HadoopMapReduceWriteConfigUtil[K, V: ClassTag](conf: SerializableConfigura
379379
private def getOutputFormat(): NewOutputFormat[K, V] = {
380380
require(outputFormat != null, "Must call initOutputFormat first.")
381381

382-
outputFormat.newInstance()
382+
outputFormat.getConstructor().newInstance()
383383
}
384384

385385
// --------------------------------------------------------------------------

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -179,7 +179,7 @@ private[spark] class MetricsSystem private (
179179
sourceConfigs.foreach { kv =>
180180
val classPath = kv._2.getProperty("class")
181181
try {
182-
val source = Utils.classForName(classPath).newInstance()
182+
val source = Utils.classForName(classPath).getConstructor().newInstance()
183183
registerSource(source.asInstanceOf[Source])
184184
} catch {
185185
case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e)

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@ private[spark] class BinaryFileRDD[T](
4141
// traversing a large number of directories and files. Parallelize it.
4242
conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS,
4343
Runtime.getRuntime.availableProcessors().toString)
44-
val inputFormat = inputFormatClass.newInstance
44+
val inputFormat = inputFormatClass.getConstructor().newInstance()
4545
inputFormat match {
4646
case configurable: Configurable =>
4747
configurable.setConf(conf)

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -120,7 +120,7 @@ class NewHadoopRDD[K, V](
120120
}
121121

122122
override def getPartitions: Array[Partition] = {
123-
val inputFormat = inputFormatClass.newInstance
123+
val inputFormat = inputFormatClass.getConstructor().newInstance()
124124
inputFormat match {
125125
case configurable: Configurable =>
126126
configurable.setConf(_conf)
@@ -183,7 +183,7 @@ class NewHadoopRDD[K, V](
183183
}
184184
}
185185

186-
private val format = inputFormatClass.newInstance
186+
private val format = inputFormatClass.getConstructor().newInstance()
187187
format match {
188188
case configurable: Configurable =>
189189
configurable.setConf(conf)

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ private[spark] class WholeTextFileRDD(
4444
// traversing a large number of directories and files. Parallelize it.
4545
conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS,
4646
Runtime.getRuntime.availableProcessors().toString)
47-
val inputFormat = inputFormatClass.newInstance
47+
val inputFormat = inputFormatClass.getConstructor().newInstance()
4848
inputFormat match {
4949
case configurable: Configurable =>
5050
configurable.setConf(conf)

0 commit comments

Comments
 (0)