Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ object SparkEnv extends Logging {
// SparkConf, then one taking no arguments
try {
cls.getConstructor(classOf[SparkConf], java.lang.Boolean.TYPE)
.newInstance(conf, new java.lang.Boolean(isDriver))
.newInstance(conf, java.lang.Boolean.valueOf(isDriver))
.asInstanceOf[T]
} catch {
case _: NoSuchMethodException =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,8 @@ private[python] object Converter extends Logging {
defaultConverter: Converter[Any, Any]): Converter[Any, Any] = {
converterClass.map { cc =>
Try {
val c = Utils.classForName(cc).newInstance().asInstanceOf[Converter[Any, Any]]
val c = Utils.classForName(cc).getConstructor().
newInstance().asInstanceOf[Converter[Any, Any]]
logInfo(s"Loaded converter: $cc")
c
} match {
Expand Down
6 changes: 3 additions & 3 deletions core/src/main/scala/org/apache/spark/api/r/SerDe.scala
Original file line number Diff line number Diff line change
Expand Up @@ -74,9 +74,9 @@ private[spark] object SerDe {
jvmObjectTracker: JVMObjectTracker): Object = {
dataType match {
case 'n' => null
case 'i' => new java.lang.Integer(readInt(dis))
case 'd' => new java.lang.Double(readDouble(dis))
case 'b' => new java.lang.Boolean(readBoolean(dis))
case 'i' => java.lang.Integer.valueOf(readInt(dis))
case 'd' => java.lang.Double.valueOf(readDouble(dis))
case 'b' => java.lang.Boolean.valueOf(readBoolean(dis))
case 'c' => readString(dis)
case 'e' => readMap(dis, jvmObjectTracker)
case 'r' => readBytes(dis)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -829,7 +829,7 @@ private[spark] class SparkSubmit extends Logging {
}

val app: SparkApplication = if (classOf[SparkApplication].isAssignableFrom(mainClass)) {
mainClass.newInstance().asInstanceOf[SparkApplication]
mainClass.getConstructor().newInstance().asInstanceOf[SparkApplication]
} else {
// SPARK-4170
if (classOf[scala.App].isAssignableFrom(mainClass)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ class HadoopMapReduceCommitProtocol(
private def stagingDir = new Path(path, ".spark-staging-" + jobId)

protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = {
val format = context.getOutputFormatClass.newInstance()
val format = context.getOutputFormatClass.getConstructor().newInstance()
// If OutputFormat is Configurable, we should set conf to it.
format match {
case c: Configurable => c.setConf(context.getConfiguration)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf)
private def getOutputFormat(): OutputFormat[K, V] = {
require(outputFormat != null, "Must call initOutputFormat first.")

outputFormat.newInstance()
outputFormat.getConstructor().newInstance()
}

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

outputFormat.newInstance()
outputFormat.getConstructor().newInstance()
}

// --------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ private[spark] class MetricsSystem private (
sourceConfigs.foreach { kv =>
val classPath = kv._2.getProperty("class")
try {
val source = Utils.classForName(classPath).newInstance()
val source = Utils.classForName(classPath).getConstructor().newInstance()
registerSource(source.asInstanceOf[Source])
} catch {
case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ private[spark] class BinaryFileRDD[T](
// traversing a large number of directories and files. Parallelize it.
conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS,
Runtime.getRuntime.availableProcessors().toString)
val inputFormat = inputFormatClass.newInstance
val inputFormat = inputFormatClass.getConstructor().newInstance()
inputFormat match {
case configurable: Configurable =>
configurable.setConf(conf)
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ class NewHadoopRDD[K, V](
}

override def getPartitions: Array[Partition] = {
val inputFormat = inputFormatClass.newInstance
val inputFormat = inputFormatClass.getConstructor().newInstance()
inputFormat match {
case configurable: Configurable =>
configurable.setConf(_conf)
Expand Down Expand Up @@ -183,7 +183,7 @@ class NewHadoopRDD[K, V](
}
}

private val format = inputFormatClass.newInstance
private val format = inputFormatClass.getConstructor().newInstance()
format match {
case configurable: Configurable =>
configurable.setConf(conf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ private[spark] class WholeTextFileRDD(
// traversing a large number of directories and files. Parallelize it.
conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS,
Runtime.getRuntime.availableProcessors().toString)
val inputFormat = inputFormatClass.newInstance
val inputFormat = inputFormatClass.getConstructor().newInstance()
inputFormat match {
case configurable: Configurable =>
configurable.setConf(conf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,8 @@ class KryoSerializer(conf: SparkConf)
.foreach { className => kryo.register(Class.forName(className, true, classLoader)) }
// Allow the user to register their own classes by setting spark.kryo.registrator.
userRegistrators
.map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator])
.map(Class.forName(_, true, classLoader).getConstructor().
newInstance().asInstanceOf[KryoRegistrator])
.foreach { reg => reg.registerClasses(kryo) }
// scalastyle:on classforname
} catch {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ private[spark] class BlockManager(
val priorityClass = conf.get(
"spark.storage.replication.policy", classOf[RandomBlockReplicationPolicy].getName)
val clazz = Utils.classForName(priorityClass)
val ret = clazz.newInstance.asInstanceOf[BlockReplicationPolicy]
val ret = clazz.getConstructor().newInstance().asInstanceOf[BlockReplicationPolicy]
logInfo(s"Using $priorityClass for block replication policy")
ret
}
Expand Down
3 changes: 2 additions & 1 deletion core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2430,7 +2430,8 @@ private[spark] object Utils extends Logging {
"org.apache.spark.security.ShellBasedGroupsMappingProvider")
if (groupProviderClassName != "") {
try {
val groupMappingServiceProvider = classForName(groupProviderClassName).newInstance.
val groupMappingServiceProvider = classForName(groupProviderClassName).
getConstructor().newInstance().
asInstanceOf[org.apache.spark.security.GroupMappingServiceProvider]
val currentUserGroups = groupMappingServiceProvider.getGroups(username)
return currentUserGroups
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/org/apache/spark/FileSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
sc = new SparkContext("local", "test")
val objs = sc.makeRDD(1 to 3).map { x =>
val loader = Thread.currentThread().getContextClassLoader
Class.forName(className, true, loader).newInstance()
Class.forName(className, true, loader).getConstructor().newInstance()
}
val outputDir = new File(tempDir, "output").getAbsolutePath
objs.saveAsObjectFile(outputDir)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -574,7 +574,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
}

test("saveNewAPIHadoopFile should call setConf if format is configurable") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(1))))
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(1))))

// No error, non-configurable formats still work
pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored")
Expand All @@ -591,14 +591,14 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
test("The JobId on the driver and executors should be the same during the commit") {
// Create more than one rdd to mimic stageId not equal to rddId
val pairs = sc.parallelize(Array((1, 2), (2, 3)), 2)
.map { p => (new Integer(p._1 + 1), new Integer(p._2 + 1)) }
.map { p => (Integer.valueOf(p._1 + 1), Integer.valueOf(p._2 + 1)) }
.filter { p => p._1 > 0 }
pairs.saveAsNewAPIHadoopFile[YetAnotherFakeFormat]("ignored")
assert(JobID.jobid != -1)
}

test("saveAsHadoopFile should respect configured output committers") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(1))))
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(1))))
val conf = new JobConf()
conf.setOutputCommitter(classOf[FakeOutputCommitter])

Expand All @@ -610,7 +610,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
}

test("failure callbacks should be called before calling writer.close() in saveNewAPIHadoopFile") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1)

FakeWriterWithCallback.calledBy = ""
FakeWriterWithCallback.exception = null
Expand All @@ -625,7 +625,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
}

test("failure callbacks should be called before calling writer.close() in saveAsHadoopFile") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1)
val conf = new JobConf()

FakeWriterWithCallback.calledBy = ""
Expand All @@ -643,7 +643,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {

test("saveAsNewAPIHadoopDataset should support invalid output paths when " +
"there are no files to be committed to an absolute output location") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1)

def saveRddWithPath(path: String): Unit = {
val job = NewJob.getInstance(new Configuration(sc.hadoopConfiguration))
Expand Down Expand Up @@ -671,7 +671,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
// for non-null invalid paths.
test("saveAsHadoopDataset should respect empty output directory when " +
"there are no files to be committed to an absolute output location") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1)

val conf = new JobConf()
conf.setOutputKeyClass(classOf[Integer])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,7 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local
// jar.
sc = new SparkContext("local", "test", conf)
val rdd = sc.parallelize(Seq(1), 1).map { _ =>
val exc = excClass.newInstance().asInstanceOf[Exception]
val exc = excClass.getConstructor().newInstance().asInstanceOf[Exception]
throw exc
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ class AccumulatorV2Suite extends SparkFunSuite {
assert(acc.avg == 0.5)

// Also test add using non-specialized add function
acc.add(new java.lang.Long(2))
acc.add(java.lang.Long.valueOf(2))
assert(acc.count == 3)
assert(acc.sum == 3)
assert(acc.avg == 1.0)
Expand Down Expand Up @@ -73,7 +73,7 @@ class AccumulatorV2Suite extends SparkFunSuite {
assert(acc.avg == 0.5)

// Also test add using non-specialized add function
acc.add(new java.lang.Double(2.0))
acc.add(java.lang.Double.valueOf(2.0))
assert(acc.count == 3)
assert(acc.sum == 3.0)
assert(acc.avg == 1.0)
Expand All @@ -96,7 +96,7 @@ class AccumulatorV2Suite extends SparkFunSuite {
assert(acc.value.contains(0.0))
assert(!acc.isZero)

acc.add(new java.lang.Double(1.0))
acc.add(java.lang.Double.valueOf(1.0))

val acc2 = acc.copyAndReset()
assert(acc2.value.isEmpty)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import java.net.URLClassLoader
import scala.collection.JavaConverters._

import org.scalatest.Matchers
import org.scalatest.Matchers._

import org.apache.spark.{SparkContext, SparkException, SparkFunSuite, TestUtils}

Expand All @@ -46,10 +45,10 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
test("child first") {
val parentLoader = new URLClassLoader(urls2, null)
val classLoader = new ChildFirstURLClassLoader(urls, parentLoader)
val fakeClass = classLoader.loadClass("FakeClass2").newInstance()
val fakeClass = classLoader.loadClass("FakeClass2").getConstructor().newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "1")
val fakeClass2 = classLoader.loadClass("FakeClass2").newInstance()
val fakeClass2 = classLoader.loadClass("FakeClass2").getConstructor().newInstance()
assert(fakeClass.getClass === fakeClass2.getClass)
classLoader.close()
parentLoader.close()
Expand All @@ -58,10 +57,10 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
test("parent first") {
val parentLoader = new URLClassLoader(urls2, null)
val classLoader = new MutableURLClassLoader(urls, parentLoader)
val fakeClass = classLoader.loadClass("FakeClass1").newInstance()
val fakeClass = classLoader.loadClass("FakeClass1").getConstructor().newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "2")
val fakeClass2 = classLoader.loadClass("FakeClass1").newInstance()
val fakeClass2 = classLoader.loadClass("FakeClass1").getConstructor().newInstance()
assert(fakeClass.getClass === fakeClass2.getClass)
classLoader.close()
parentLoader.close()
Expand All @@ -70,7 +69,7 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
test("child first can fall back") {
val parentLoader = new URLClassLoader(urls2, null)
val classLoader = new ChildFirstURLClassLoader(urls, parentLoader)
val fakeClass = classLoader.loadClass("FakeClass3").newInstance()
val fakeClass = classLoader.loadClass("FakeClass3").getConstructor().newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "2")
classLoader.close()
Expand All @@ -81,7 +80,7 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
val parentLoader = new URLClassLoader(urls2, null)
val classLoader = new ChildFirstURLClassLoader(urls, parentLoader)
intercept[java.lang.ClassNotFoundException] {
classLoader.loadClass("FakeClassDoesNotExist").newInstance()
classLoader.loadClass("FakeClassDoesNotExist").getConstructor().newInstance()
}
classLoader.close()
parentLoader.close()
Expand Down Expand Up @@ -137,7 +136,7 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
sc.makeRDD(1 to 5, 2).mapPartitions { x =>
val loader = Thread.currentThread().getContextClassLoader
// scalastyle:off classforname
Class.forName(className, true, loader).newInstance()
Class.forName(className, true, loader).getConstructor().newInstance()
// scalastyle:on classforname
Seq().iterator
}.count()
Expand Down
32 changes: 16 additions & 16 deletions core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -92,14 +92,14 @@ class SizeEstimatorSuite
}

test("primitive wrapper objects") {
assertResult(16)(SizeEstimator.estimate(new java.lang.Boolean(true)))
assertResult(16)(SizeEstimator.estimate(new java.lang.Byte("1")))
assertResult(16)(SizeEstimator.estimate(new java.lang.Character('1')))
assertResult(16)(SizeEstimator.estimate(new java.lang.Short("1")))
assertResult(16)(SizeEstimator.estimate(new java.lang.Integer(1)))
assertResult(24)(SizeEstimator.estimate(new java.lang.Long(1)))
assertResult(16)(SizeEstimator.estimate(new java.lang.Float(1.0)))
assertResult(24)(SizeEstimator.estimate(new java.lang.Double(1.0d)))
assertResult(16)(SizeEstimator.estimate(java.lang.Boolean.TRUE))
assertResult(16)(SizeEstimator.estimate(java.lang.Byte.valueOf("1")))
assertResult(16)(SizeEstimator.estimate(java.lang.Character.valueOf('1')))
assertResult(16)(SizeEstimator.estimate(java.lang.Short.valueOf("1")))
assertResult(16)(SizeEstimator.estimate(java.lang.Integer.valueOf(1)))
assertResult(24)(SizeEstimator.estimate(java.lang.Long.valueOf(1)))
assertResult(16)(SizeEstimator.estimate(java.lang.Float.valueOf(1.0f)))
assertResult(24)(SizeEstimator.estimate(java.lang.Double.valueOf(1.0)))
}

test("class field blocks rounding") {
Expand Down Expand Up @@ -202,14 +202,14 @@ class SizeEstimatorSuite
assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh")))

// primitive wrapper classes
assertResult(24)(SizeEstimator.estimate(new java.lang.Boolean(true)))
assertResult(24)(SizeEstimator.estimate(new java.lang.Byte("1")))
assertResult(24)(SizeEstimator.estimate(new java.lang.Character('1')))
assertResult(24)(SizeEstimator.estimate(new java.lang.Short("1")))
assertResult(24)(SizeEstimator.estimate(new java.lang.Integer(1)))
assertResult(24)(SizeEstimator.estimate(new java.lang.Long(1)))
assertResult(24)(SizeEstimator.estimate(new java.lang.Float(1.0)))
assertResult(24)(SizeEstimator.estimate(new java.lang.Double(1.0d)))
assertResult(24)(SizeEstimator.estimate(java.lang.Boolean.TRUE))
assertResult(24)(SizeEstimator.estimate(java.lang.Byte.valueOf("1")))
assertResult(24)(SizeEstimator.estimate(java.lang.Character.valueOf('1')))
assertResult(24)(SizeEstimator.estimate(java.lang.Short.valueOf("1")))
assertResult(24)(SizeEstimator.estimate(java.lang.Integer.valueOf(1)))
assertResult(24)(SizeEstimator.estimate(java.lang.Long.valueOf(1)))
assertResult(24)(SizeEstimator.estimate(java.lang.Float.valueOf(1.0f)))
assertResult(24)(SizeEstimator.estimate(java.lang.Double.valueOf(1.0)))
}

test("class field blocks rounding on 64-bit VM without useCompressedOops") {
Expand Down
Loading