diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 72123f2232..66038eeaea 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 { // 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 => diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index 6259bead3e..2ab8add63e 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -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 { diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index 537ab57f96..6e0a3f6398 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -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) 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 88df7324a3..0fc8c9bd78 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -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)) { diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 3e60c50ada..7477e03bfa 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -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) diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index 9ebd0aa301..3a58ea8169 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -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() } // -------------------------------------------------------------------------- @@ -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() } // -------------------------------------------------------------------------- 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 3457a26322..bb7b434e9a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -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) diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index a14bad47df..039dbcbd5e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -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) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2d66d25ba3..483de28d92 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -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) @@ -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) diff --git a/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala index 9f3d0745c3..eada762b99 100644 --- a/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala @@ -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) 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 72427dd6ce..218c84352c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -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 { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e35dd72521..edae2f95fc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -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 } 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 93b5826f8a..a07eee6ad8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -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 diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 34efcdf4bc..df04a5ea1d 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -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) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 47af5c3320..0ec359d1c9 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -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") @@ -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]) @@ -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 @@ -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 = "" @@ -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)) @@ -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]) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 1bddba8f6c..f8eb8bd71c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -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 } diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala index 621399af73..172bebbfec 100644 --- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -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) @@ -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) @@ -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) 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 f6ac89fc27..8d844bd087 100644 --- a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -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} @@ -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() @@ -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() @@ -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() @@ -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() @@ -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() diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 2695295d45..63f9f82adf 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -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") { @@ -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") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index 65bf857e22..46a05e2ba7 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import java.lang.{Float => JFloat, Integer => JInteger} +import java.lang.{Float => JFloat} import java.util.{Arrays, Comparator} import org.apache.spark.SparkFunSuite @@ -48,7 +48,7 @@ class SorterSuite extends SparkFunSuite with Logging { // alternate. Keys are random doubles, values are ordinals from 0 to length. val keys = Array.tabulate[Double](5000) { i => rand.nextDouble() } val keyValueArray = Array.tabulate[Number](10000) { i => - if (i % 2 == 0) keys(i / 2) else new Integer(i / 2) + if (i % 2 == 0) keys(i / 2) else Integer.valueOf(i / 2) } // Map from generated keys to values, to verify correctness later @@ -112,7 +112,7 @@ class SorterSuite extends SparkFunSuite with Logging { // Test our key-value pairs where each element is a Tuple2[Float, Integer]. val kvTuples = Array.tabulate(numElements) { i => - (new JFloat(rand.nextFloat()), new JInteger(i)) + (JFloat.valueOf(rand.nextFloat()), Integer.valueOf(i)) } val kvTupleArray = new Array[AnyRef](numElements) @@ -167,23 +167,23 @@ class SorterSuite extends SparkFunSuite with Logging { val ints = Array.fill(numElements)(rand.nextInt()) val intObjects = { - val data = new Array[JInteger](numElements) + val data = new Array[Integer](numElements) var i = 0 while (i < numElements) { - data(i) = new JInteger(ints(i)) + data(i) = Integer.valueOf(ints(i)) i += 1 } data } - val intObjectArray = new Array[JInteger](numElements) + val intObjectArray = new Array[Integer](numElements) val prepareIntObjectArray = () => { System.arraycopy(intObjects, 0, intObjectArray, 0, numElements) } runExperiment("Java Arrays.sort() on non-primitive int array")({ - Arrays.sort(intObjectArray, new Comparator[JInteger] { - override def compare(x: JInteger, y: JInteger): Int = x.compareTo(y) + Arrays.sort(intObjectArray, new Comparator[Integer] { + override def compare(x: Integer, y: Integer): Int = x.compareTo(y) }) }, prepareIntObjectArray) diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index d5956ea320..d570630c1a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -78,7 +78,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { private def generateTestData(size: Long, rand: => Long): (Array[JLong], LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size))(0) - (ref.map(i => new JLong(i)), new LongArray(MemoryBlock.fromLongArray(extended))) + (ref.map(i => JLong.valueOf(i)), new LongArray(MemoryBlock.fromLongArray(extended))) } private def generateKeyPrefixTestData(size: Long, rand: => Long): (LongArray, LongArray) = { diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 4fea2cb969..8d6cca8e48 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -508,7 +508,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val union2 = spark.read.format("avro").load(testAvro).select("union_float_double").collect() assert( union2 - .map(x => new java.lang.Double(x(0).toString)) + .map(x => java.lang.Double.valueOf(x(0).toString)) .exists(p => Math.abs(p - Math.PI) < 0.001)) val fixed = spark.read.format("avro").load(testAvro).select("fixed3").collect() diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 28c9853bfe..5034bd73d6 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -510,7 +510,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") // So that the driver does not pull too much data - .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1)) + .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, java.lang.Integer.valueOf(1)) // If buffer config is not set, set it to reasonable value to work around // buffer issues (see KAFKA-3135) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index ba4009ef08..224f41a683 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -70,7 +70,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( @transient private var kc: Consumer[K, V] = null def consumer(): Consumer[K, V] = this.synchronized { if (null == kc) { - kc = consumerStrategy.onStart(currentOffsets.mapValues(l => new java.lang.Long(l)).asJava) + kc = consumerStrategy.onStart(currentOffsets.mapValues(l => java.lang.Long.valueOf(l)).asJava) } kc } diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index a0ac26a34d..d985f8ca1e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -256,7 +256,7 @@ class GeneralMLWriter(stage: PipelineStage) extends MLWriter with Logging { s"Multiple writers found for $source+$stageName, try using the class name of the writer") } if (classOf[MLWriterFormat].isAssignableFrom(writerCls)) { - val writer = writerCls.newInstance().asInstanceOf[MLWriterFormat] + val writer = writerCls.getConstructor().newInstance().asInstanceOf[MLWriterFormat] writer.write(path, sparkSession, optionMap, stage) } else { throw new SparkException(s"ML source $source is not a valid MLWriterFormat") diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala index bdceba7887..8371c33a20 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala @@ -31,7 +31,7 @@ class MatrixUDTSuite extends SparkFunSuite { val sm3 = dm3.toSparse for (m <- Seq(dm1, dm2, dm3, sm1, sm2, sm3)) { - val udt = UDTRegistration.getUDTFor(m.getClass.getName).get.newInstance() + val udt = UDTRegistration.getUDTFor(m.getClass.getName).get.getConstructor().newInstance() .asInstanceOf[MatrixUDT] assert(m === udt.deserialize(udt.serialize(m))) assert(udt.typeName == "matrix") diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 6ddb12cb76..67c64f762b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -31,7 +31,7 @@ class VectorUDTSuite extends SparkFunSuite { val sv2 = Vectors.sparse(2, Array(1), Array(2.0)) for (v <- Seq(dv1, dv2, sv1, sv2)) { - val udt = UDTRegistration.getUDTFor(v.getClass.getName).get.newInstance() + val udt = UDTRegistration.getUDTFor(v.getClass.getName).get.getConstructor().newInstance() .asInstanceOf[VectorUDT] assert(v === udt.deserialize(udt.serialize(v))) assert(udt.typeName == "vector") diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index e5e2094368..ac528ecb82 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -126,7 +126,7 @@ class ExecutorClassLoaderSuite test("child first") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true) - val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass2").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "1") } @@ -134,7 +134,7 @@ class ExecutorClassLoaderSuite test("parent first") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, false) - val fakeClass = classLoader.loadClass("ReplFakeClass1").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass1").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") } @@ -142,7 +142,7 @@ class ExecutorClassLoaderSuite test("child first can fall back") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true) - val fakeClass = classLoader.loadClass("ReplFakeClass3").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass3").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") } @@ -151,7 +151,7 @@ class ExecutorClassLoaderSuite val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true) intercept[java.lang.ClassNotFoundException] { - classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() + classLoader.loadClass("ReplFakeClassDoesNotExist").getConstructor().newInstance() } } @@ -202,11 +202,11 @@ class ExecutorClassLoaderSuite val classLoader = new ExecutorClassLoader(new SparkConf(), env, "spark://localhost:1234", getClass().getClassLoader(), false) - val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass2").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "1") intercept[java.lang.ClassNotFoundException] { - classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() + classLoader.loadClass("ReplFakeClassDoesNotExist").getConstructor().newInstance() } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala index 4ed285230f..7d15f0e2fb 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala @@ -107,7 +107,7 @@ private[spark] class SchedulerExtensionServices extends SchedulerExtensionServic services = sparkContext.conf.get(SCHEDULER_SERVICES).map { sClass => val instance = Utils.classForName(sClass) - .newInstance() + .getConstructor().newInstance() .asInstanceOf[SchedulerExtensionService] // bind this service instance.start(binding) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 8ef8b2be69..311060e596 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -73,10 +73,10 @@ object JavaTypeInference { : (DataType, Boolean) = { typeToken.getRawType match { case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => - (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) + (c.getAnnotation(classOf[SQLUserDefinedType]).udt().getConstructor().newInstance(), true) case c: Class[_] if UDTRegistration.exists(c.getName) => - val udt = UDTRegistration.getUDTFor(c.getName).get.newInstance() + val udt = UDTRegistration.getUDTFor(c.getName).get.getConstructor().newInstance() .asInstanceOf[UserDefinedType[_ >: Null]] (udt, true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 912744eab6..64ea236532 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -357,7 +357,8 @@ object ScalaReflection extends ScalaReflection { ) case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). + getConstructor().newInstance() val obj = NewInstance( udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), Nil, @@ -365,8 +366,8 @@ object ScalaReflection extends ScalaReflection { Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] val obj = NewInstance( udt.getClass, Nil, @@ -601,7 +602,7 @@ object ScalaReflection extends ScalaReflection { case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => val udt = getClassFromType(t) - .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + .getAnnotation(classOf[SQLUserDefinedType]).udt().getConstructor().newInstance() val obj = NewInstance( udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), Nil, @@ -609,8 +610,8 @@ object ScalaReflection extends ScalaReflection { Invoke(obj, "serialize", udt, inputObject :: Nil) case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] val obj = NewInstance( udt.getClass, Nil, @@ -721,11 +722,12 @@ object ScalaReflection extends ScalaReflection { // Null type would wrongly match the first of them, which is Option as of now case t if t <:< definitions.NullTpe => Schema(NullType, nullable = true) case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). + getConstructor().newInstance() Schema(udt, nullable = true) case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] Schema(udt, nullable = true) case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index c11b444212..b6771ec4df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1134,7 +1134,8 @@ class SessionCatalog( if (clsForUDAF.isAssignableFrom(clazz)) { val cls = Utils.classForName("org.apache.spark.sql.execution.aggregate.ScalaUDAF") val e = cls.getConstructor(classOf[Seq[Expression]], clsForUDAF, classOf[Int], classOf[Int]) - .newInstance(input, clazz.newInstance().asInstanceOf[Object], Int.box(1), Int.box(1)) + .newInstance(input, + clazz.getConstructor().newInstance().asInstanceOf[Object], Int.box(1), Int.box(1)) .asInstanceOf[ImplicitCastInputTypes] // Check input argument size diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index b868a0f4fa..7c8f7cd431 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1305,7 +1305,7 @@ object CodeGenerator extends Logging { throw new CompileException(msg, e.getLocation) } - (evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass], maxCodeSize) + (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], maxCodeSize) } /** 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 e53628d11c..33fc4b9480 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 @@ -180,7 +180,7 @@ object DataType { ("pyClass", _), ("sqlType", _), ("type", JString("udt"))) => - Utils.classForName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] + Utils.classForName(udtClass).getConstructor().newInstance().asInstanceOf[UserDefinedType[_]] // Python UDT case JSortedObject( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index e9b100b3b3..be8fd90c4c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -128,13 +128,13 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(-3.7f, "primitive float") encodeDecodeTest(-3.7, "primitive double") - encodeDecodeTest(new java.lang.Boolean(false), "boxed boolean") - encodeDecodeTest(new java.lang.Byte(-3.toByte), "boxed byte") - encodeDecodeTest(new java.lang.Short(-3.toShort), "boxed short") - encodeDecodeTest(new java.lang.Integer(-3), "boxed int") - encodeDecodeTest(new java.lang.Long(-3L), "boxed long") - encodeDecodeTest(new java.lang.Float(-3.7f), "boxed float") - encodeDecodeTest(new java.lang.Double(-3.7), "boxed double") + encodeDecodeTest(java.lang.Boolean.FALSE, "boxed boolean") + encodeDecodeTest(java.lang.Byte.valueOf(-3: Byte), "boxed byte") + encodeDecodeTest(java.lang.Short.valueOf(-3: Short), "boxed short") + encodeDecodeTest(java.lang.Integer.valueOf(-3), "boxed int") + encodeDecodeTest(java.lang.Long.valueOf(-3L), "boxed long") + encodeDecodeTest(java.lang.Float.valueOf(-3.7f), "boxed float") + encodeDecodeTest(java.lang.Double.valueOf(-3.7), "boxed double") encodeDecodeTest(BigDecimal("32131413.211321313"), "scala decimal") encodeDecodeTest(new java.math.BigDecimal("231341.23123"), "java decimal") @@ -224,7 +224,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes productTest( RepeatedData( Seq(1, 2), - Seq(new Integer(1), null, new Integer(2)), + Seq(Integer.valueOf(1), null, Integer.valueOf(2)), Map(1 -> 2L), Map(1 -> null), PrimitiveData(1, 1, 1, 1, 1, 1, true))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index d145fd0aab..16842c1bcc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -307,7 +307,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val conf = new SparkConf() Seq(true, false).foreach { useKryo => val serializer = if (useKryo) new KryoSerializer(conf) else new JavaSerializer(conf) - val expected = serializer.newInstance().serialize(new Integer(1)).array() + val expected = serializer.newInstance().serialize(Integer.valueOf(1)).array() val encodeUsingSerializer = EncodeUsingSerializer(inputObject, useKryo) checkEvaluation(encodeUsingSerializer, expected, InternalRow.fromSeq(Seq(1))) checkEvaluation(encodeUsingSerializer, null, InternalRow.fromSeq(Seq(null))) @@ -384,9 +384,9 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val conf = new SparkConf() Seq(true, false).foreach { useKryo => val serializer = if (useKryo) new KryoSerializer(conf) else new JavaSerializer(conf) - val input = serializer.newInstance().serialize(new Integer(1)).array() + val input = serializer.newInstance().serialize(Integer.valueOf(1)).array() val decodeUsingSerializer = DecodeUsingSerializer(inputObject, ClassTag(cls), useKryo) - checkEvaluation(decodeUsingSerializer, new Integer(1), InternalRow.fromSeq(Seq(input))) + checkEvaluation(decodeUsingSerializer, Integer.valueOf(1), InternalRow.fromSeq(Seq(input))) checkEvaluation(decodeUsingSerializer, null, InternalRow.fromSeq(Seq(null))) } } @@ -575,7 +575,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // NULL key test val scalaMapHasNullKey = scala.collection.Map[java.lang.Integer, String]( - null.asInstanceOf[java.lang.Integer] -> "v0", new java.lang.Integer(1) -> "v1") + null.asInstanceOf[java.lang.Integer] -> "v0", java.lang.Integer.valueOf(1) -> "v1") val javaMapHasNullKey = new java.util.HashMap[java.lang.Integer, java.lang.String]() { { put(null, "v0") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 294fce8e9a..63c7b42978 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -41,9 +41,9 @@ class PercentileSuite extends SparkFunSuite { val buffer = new OpenHashMap[AnyRef, Long]() assert(compareEquals(agg.deserialize(agg.serialize(buffer)), buffer)) - // Check non-empty buffer serializa and deserialize. + // Check non-empty buffer serialize and deserialize. data.foreach { key => - buffer.changeValue(new Integer(key), 1L, _ + 1L) + buffer.changeValue(Integer.valueOf(key), 1L, _ + 1L) } assert(compareEquals(agg.deserialize(agg.serialize(buffer)), buffer)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 02ffc94018..df18623e42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -194,7 +194,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val ds = cls.newInstance().asInstanceOf[DataSourceV2] + val ds = cls.getConstructor().newInstance().asInstanceOf[DataSourceV2] if (ds.isInstanceOf[BatchReadSupportProvider]) { val sessionOptions = DataSourceV2Utils.extractSessionConfigs( ds = ds, conf = sparkSession.sessionState.conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 5a28870f5d..1b4998f94b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -243,7 +243,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val source = cls.newInstance().asInstanceOf[DataSourceV2] + val source = cls.getConstructor().newInstance().asInstanceOf[DataSourceV2] source match { case provider: BatchWriteSupportProvider => val sessionOptions = DataSourceV2Utils.extractSessionConfigs( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 71f967a59d..c0727e844a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -1144,7 +1144,7 @@ object SparkSession extends Logging { val extensionConfClassName = extensionOption.get try { val extensionConfClass = Utils.classForName(extensionConfClassName) - val extensionConf = extensionConfClass.newInstance() + val extensionConf = extensionConfClass.getConstructor().newInstance() .asInstanceOf[SparkSessionExtensions => Unit] extensionConf(extensions) } catch { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index aa3a6c3bf1..84da097be5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -670,7 +670,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends throw new AnalysisException(s"It is invalid to implement multiple UDF interfaces, UDF class $className") } else { try { - val udf = clazz.newInstance() + val udf = clazz.getConstructor().newInstance() val udfReturnType = udfInterfaces(0).getActualTypeArguments.last var returnType = returnDataType if (returnType == null) { @@ -727,7 +727,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends if (!classOf[UserDefinedAggregateFunction].isAssignableFrom(clazz)) { throw new AnalysisException(s"class $className doesn't implement interface UserDefinedAggregateFunction") } - val udaf = clazz.newInstance().asInstanceOf[UserDefinedAggregateFunction] + val udaf = clazz.getConstructor().newInstance().asInstanceOf[UserDefinedAggregateFunction] register(name, udaf) } catch { case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class ${className}, please make sure it is on the classpath") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index af20764f9a..becb05cf72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -111,7 +111,7 @@ private[sql] object SQLUtils extends Logging { private[this] def doConversion(data: Object, dataType: DataType): Object = { data match { case d: java.lang.Double if dataType == FloatType => - new java.lang.Float(d) + java.lang.Float.valueOf(d.toFloat) // Scala Map is the only allowed external type of map type in Row. case m: java.util.Map[_, _] => m.asScala case _ => data diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 823dc0d5ed..e2cd40906f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -231,7 +231,8 @@ case class AlterTableAddColumnsCommand( } if (DDLUtils.isDatasourceTable(catalogTable)) { - DataSource.lookupDataSource(catalogTable.provider.get, conf).newInstance() match { + DataSource.lookupDataSource(catalogTable.provider.get, conf). + getConstructor().newInstance() match { // For datasource table, this command can only support the following File format. // TextFileFormat only default to one column "value" // Hive type is already considered as hive serde table, so the logic will not diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index ce3bc3dd48..795a6d0b6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -204,7 +204,7 @@ case class DataSource( /** Returns the name and schema of the source that can be used to continually read data. */ private def sourceSchema(): SourceInfo = { - providingClass.newInstance() match { + providingClass.getConstructor().newInstance() match { case s: StreamSourceProvider => val (name, schema) = s.sourceSchema( sparkSession.sqlContext, userSpecifiedSchema, className, caseInsensitiveOptions) @@ -250,7 +250,7 @@ case class DataSource( /** Returns a source that can be used to continually read data. */ def createSource(metadataPath: String): Source = { - providingClass.newInstance() match { + providingClass.getConstructor().newInstance() match { case s: StreamSourceProvider => s.createSource( sparkSession.sqlContext, @@ -279,7 +279,7 @@ case class DataSource( /** Returns a sink that can be used to continually write data. */ def createSink(outputMode: OutputMode): Sink = { - providingClass.newInstance() match { + providingClass.getConstructor().newInstance() match { case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, caseInsensitiveOptions, partitionColumns, outputMode) @@ -310,7 +310,7 @@ case class DataSource( * that files already exist, we don't need to check them again. */ def resolveRelation(checkFilesExist: Boolean = true): BaseRelation = { - val relation = (providingClass.newInstance(), userSpecifiedSchema) match { + val relation = (providingClass.getConstructor().newInstance(), userSpecifiedSchema) match { // TODO: Throw when too much is given. case (dataSource: SchemaRelationProvider, Some(schema)) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions, schema) @@ -479,7 +479,7 @@ case class DataSource( throw new AnalysisException("Cannot save interval data type into external storage.") } - providingClass.newInstance() match { + providingClass.getConstructor().newInstance() match { case dataSource: CreatableRelationProvider => dataSource.createRelation( sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) @@ -516,7 +516,7 @@ case class DataSource( throw new AnalysisException("Cannot save interval data type into external storage.") } - providingClass.newInstance() match { + providingClass.getConstructor().newInstance() match { case dataSource: CreatableRelationProvider => SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode) case format: FileFormat => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala index 1723596de1..530d836d9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala @@ -50,7 +50,7 @@ object DriverRegistry extends Logging { } else { synchronized { if (wrapperMap.get(className).isEmpty) { - val wrapper = new DriverWrapper(cls.newInstance().asInstanceOf[Driver]) + val wrapper = new DriverWrapper(cls.getConstructor().newInstance().asInstanceOf[Driver]) DriverManager.registerDriver(wrapper) wrapperMap(className) = wrapper logTrace(s"Wrapper for $className registered") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index d3313b8a31..7d785aa09c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -213,7 +213,7 @@ object StateStoreProvider { */ def create(providerClassName: String): StateStoreProvider = { val providerClass = Utils.classForName(providerClassName) - providerClass.newInstance().asInstanceOf[StateStoreProvider] + providerClass.getConstructor().newInstance().asInstanceOf[StateStoreProvider] } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 20c8430577..bf6021e692 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -158,7 +158,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo "read files of Hive data source directly.") } - val ds = DataSource.lookupDataSource(source, sparkSession.sqlContext.conf).newInstance() + val ds = DataSource.lookupDataSource(source, sparkSession.sqlContext.conf). + getConstructor().newInstance() // We need to generate the V1 data source so we can pass it to the V2 relation as a shim. // We can't be sure at this point whether we'll actually want to use V2, since we don't know the // writer or whether the query is continuous. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 4a8c7fdb58..b36a8f3f6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -307,7 +307,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { val ds = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") var options = extraOptions.toMap - val sink = ds.newInstance() match { + val sink = ds.getConstructor().newInstance() match { case w: StreamingWriteSupportProvider if !disabledSources.contains(w.getClass.getCanonicalName) => val sessionOptions = DataSourceV2Utils.extractSessionConfigs( diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java index 3ab4db2a03..ca78d6489e 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java @@ -67,20 +67,20 @@ public class JavaRowSuite { public void constructSimpleRow() { Row simpleRow = RowFactory.create( byteValue, // ByteType - new Byte(byteValue), + Byte.valueOf(byteValue), shortValue, // ShortType - new Short(shortValue), + Short.valueOf(shortValue), intValue, // IntegerType - new Integer(intValue), + Integer.valueOf(intValue), longValue, // LongType - new Long(longValue), + Long.valueOf(longValue), floatValue, // FloatType - new Float(floatValue), + Float.valueOf(floatValue), doubleValue, // DoubleType - new Double(doubleValue), + Double.valueOf(doubleValue), decimalValue, // DecimalType booleanValue, // BooleanType - new Boolean(booleanValue), + Boolean.valueOf(booleanValue), stringValue, // StringType binaryValue, // BinaryType dateValue, // DateType diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java index b90224f2ae..5955eabe49 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java @@ -25,6 +25,6 @@ import org.apache.spark.sql.api.java.UDF1; public class JavaStringLength implements UDF1 { @Override public Integer call(String str) throws Exception { - return new Integer(str.length()); + return Integer.valueOf(str.length()); } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala index d635912cf7..52708f5fe4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala @@ -208,7 +208,7 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext { test("percentile_approx(col, ...), input rows contains null, with out group by") { withTempView(table) { - (1 to 1000).map(new Integer(_)).flatMap(Seq(null: Integer, _)).toDF("col") + (1 to 1000).map(Integer.valueOf(_)).flatMap(Seq(null: Integer, _)).toDF("col") .createOrReplaceTempView(table) checkAnswer( spark.sql( @@ -226,8 +226,8 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext { withTempView(table) { val rand = new java.util.Random() (1 to 1000) - .map(new Integer(_)) - .map(v => (new Integer(v % 2), v)) + .map(Integer.valueOf(_)) + .map(v => (Integer.valueOf(v % 2), v)) // Add some nulls .flatMap(Seq(_, (null: Integer, null: Integer))) .toDF("key", "value").createOrReplaceTempView(table) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index edde9bfd08..2bb18f48e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1986,7 +1986,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("SPARK-11725: correctly handle null inputs for ScalaUDF") { val df = sparkContext.parallelize(Seq( - new java.lang.Integer(22) -> "John", + java.lang.Integer.valueOf(22) -> "John", null.asInstanceOf[java.lang.Integer] -> "Lucy")).toDF("age", "name") // passing null into the UDF that could handle it @@ -2219,9 +2219,9 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("SPARK-17957: no change on nullability in FilterExec output") { val df = sparkContext.parallelize(Seq( - null.asInstanceOf[java.lang.Integer] -> new java.lang.Integer(3), - new java.lang.Integer(1) -> null.asInstanceOf[java.lang.Integer], - new java.lang.Integer(2) -> new java.lang.Integer(4))).toDF() + null.asInstanceOf[java.lang.Integer] -> java.lang.Integer.valueOf(3), + java.lang.Integer.valueOf(1) -> null.asInstanceOf[java.lang.Integer], + java.lang.Integer.valueOf(2) -> java.lang.Integer.valueOf(4))).toDF() verifyNullabilityInFilterExec(df, expr = "Rand()", expectedNonNullableColumns = Seq.empty[String]) @@ -2236,9 +2236,9 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("SPARK-17957: set nullability to false in FilterExec output") { val df = sparkContext.parallelize(Seq( - null.asInstanceOf[java.lang.Integer] -> new java.lang.Integer(3), - new java.lang.Integer(1) -> null.asInstanceOf[java.lang.Integer], - new java.lang.Integer(2) -> new java.lang.Integer(4))).toDF() + null.asInstanceOf[java.lang.Integer] -> java.lang.Integer.valueOf(3), + java.lang.Integer.valueOf(1) -> null.asInstanceOf[java.lang.Integer], + java.lang.Integer.valueOf(2) -> java.lang.Integer.valueOf(4))).toDF() verifyNullabilityInFilterExec(df, expr = "_1 + _2 * 3", expectedNonNullableColumns = Seq("_1", "_2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 82d3b22a48..75d0651037 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -697,15 +697,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("SPARK-11894: Incorrect results are returned when using null") { val nullInt = null.asInstanceOf[java.lang.Integer] - val ds1 = Seq((nullInt, "1"), (new java.lang.Integer(22), "2")).toDS() - val ds2 = Seq((nullInt, "1"), (new java.lang.Integer(22), "2")).toDS() + val ds1 = Seq((nullInt, "1"), (java.lang.Integer.valueOf(22), "2")).toDS() + val ds2 = Seq((nullInt, "1"), (java.lang.Integer.valueOf(22), "2")).toDS() checkDataset( ds1.joinWith(ds2, lit(true), "cross"), ((nullInt, "1"), (nullInt, "1")), - ((nullInt, "1"), (new java.lang.Integer(22), "2")), - ((new java.lang.Integer(22), "2"), (nullInt, "1")), - ((new java.lang.Integer(22), "2"), (new java.lang.Integer(22), "2"))) + ((nullInt, "1"), (java.lang.Integer.valueOf(22), "2")), + ((java.lang.Integer.valueOf(22), "2"), (nullInt, "1")), + ((java.lang.Integer.valueOf(22), "2"), (java.lang.Integer.valueOf(22), "2"))) } test("change encoder with compatible schema") { @@ -881,7 +881,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(ds.rdd.map(r => r.id).count === 2) assert(ds2.rdd.map(r => r.id).count === 2) - val ds3 = ds.map(g => new java.lang.Long(g.id)) + val ds3 = ds.map(g => java.lang.Long.valueOf(g.id)) assert(ds3.rdd.map(r => r).count === 2) } @@ -1499,7 +1499,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(e.getCause.isInstanceOf[NullPointerException]) withTempPath { path => - Seq(new Integer(1), null).toDF("i").write.parquet(path.getCanonicalPath) + Seq(Integer.valueOf(1), null).toDF("i").write.parquet(path.getCanonicalPath) // If the primitive values are from files, we need to do runtime null check. val ds = spark.read.parquet(path.getCanonicalPath).as[Int] intercept[NullPointerException](ds.collect()) @@ -1553,7 +1553,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val df = Seq("Amsterdam", "San Francisco", "X").toDF("city") checkAnswer(df.where('city === 'X'), Seq(Row("X"))) checkAnswer( - df.where($"city".contains(new java.lang.Character('A'))), + df.where($"city".contains(java.lang.Character.valueOf('A'))), Seq(Row("Amsterdam"))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 3b301a4f81..20dcefa7e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -413,7 +413,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { test("SPARK-25044 Verify null input handling for primitive types - with udf.register") { withTable("t") { - Seq((null, new Integer(1), "x"), ("M", null, "y"), ("N", new Integer(3), null)) + Seq((null, Integer.valueOf(1), "x"), ("M", null, "y"), ("N", Integer.valueOf(3), null)) .toDF("a", "b", "c").write.format("json").saveAsTable("t") spark.udf.register("f", (a: String, b: Int, c: Any) => a + b + c) val df = spark.sql("SELECT f(a, b, c) FROM t") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala index d4e7e362c6..3121b7e99c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala @@ -39,7 +39,7 @@ class ColumnStatsSuite extends SparkFunSuite { val columnStatsName = columnStatsClass.getSimpleName test(s"$columnStatsName: empty") { - val columnStats = columnStatsClass.newInstance() + val columnStats = columnStatsClass.getConstructor().newInstance() columnStats.collectedStatistics.zip(initialStatistics).foreach { case (actual, expected) => assert(actual === expected) } @@ -48,7 +48,7 @@ class ColumnStatsSuite extends SparkFunSuite { test(s"$columnStatsName: non-empty") { import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ - val columnStats = columnStatsClass.newInstance() + val columnStats = columnStatsClass.getConstructor().newInstance() val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) rows.foreach(columnStats.gatherStats(_, 0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala index dd74af873c..be3efed714 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala @@ -53,7 +53,8 @@ class RateSourceSuite extends StreamTest { test("microbatch in registry") { withTempDir { temp => - DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { + DataSource.lookupDataSource("rate", spark.sqlContext.conf). + getConstructor().newInstance() match { case ds: MicroBatchReadSupportProvider => val readSupport = ds.createMicroBatchReadSupport( temp.getCanonicalPath, DataSourceOptions.empty()) @@ -66,7 +67,7 @@ class RateSourceSuite extends StreamTest { test("compatible with old path in registry") { DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.RateSourceProvider", - spark.sqlContext.conf).newInstance() match { + spark.sqlContext.conf).getConstructor().newInstance() match { case ds: MicroBatchReadSupportProvider => assert(ds.isInstanceOf[RateStreamProvider]) case _ => @@ -320,7 +321,8 @@ class RateSourceSuite extends StreamTest { } test("continuous in registry") { - DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { + DataSource.lookupDataSource("rate", spark.sqlContext.conf). + getConstructor().newInstance() match { case ds: ContinuousReadSupportProvider => val readSupport = ds.createContinuousReadSupport( "", DataSourceOptions.empty()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala index 409156e5eb..635ea6fca6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala @@ -84,7 +84,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before test("backward compatibility with old path") { DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.TextSocketSourceProvider", - spark.sqlContext.conf).newInstance() match { + spark.sqlContext.conf).getConstructor().newInstance() match { case ds: MicroBatchReadSupportProvider => assert(ds.isInstanceOf[TextSocketSourceProvider]) case _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala index 4911e32255..f903c17923 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala @@ -33,7 +33,7 @@ class DataSourceV2UtilsSuite extends SparkFunSuite { conf.setConfString(s"spark.sql.$keyPrefix.config.name", "false") conf.setConfString("spark.datasource.another.config.name", "123") conf.setConfString(s"spark.datasource.$keyPrefix.", "123") - val cs = classOf[DataSourceV2WithSessionConfig].newInstance() + val cs = classOf[DataSourceV2WithSessionConfig].getConstructor().newInstance() val confs = DataSourceV2Utils.extractSessionConfigs(cs.asInstanceOf[DataSourceV2], conf) assert(confs.size == 2) assert(confs.keySet.filter(_.startsWith("spark.datasource")).size == 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 3a0e780a73..31fce46c2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -261,7 +261,7 @@ class StreamingDataSourceV2Suite extends StreamTest { ).foreach { case (source, trigger) => test(s"SPARK-25460: session options are respected in structured streaming sources - $source") { // `keyPrefix` and `shortName` are the same in this test case - val readSource = source.newInstance().shortName() + val readSource = source.getConstructor().newInstance().shortName() val writeSource = "fake-write-microbatch-continuous" val readOptionName = "optionA" @@ -299,8 +299,10 @@ class StreamingDataSourceV2Suite extends StreamTest { for ((read, write, trigger) <- cases) { testQuietly(s"stream with read format $read, write format $write, trigger $trigger") { - val readSource = DataSource.lookupDataSource(read, spark.sqlContext.conf).newInstance() - val writeSource = DataSource.lookupDataSource(write, spark.sqlContext.conf).newInstance() + val readSource = DataSource.lookupDataSource(read, spark.sqlContext.conf). + getConstructor().newInstance() + val writeSource = DataSource.lookupDataSource(write, spark.sqlContext.conf). + getConstructor().newInstance() (readSource, writeSource, trigger) match { // Valid microbatch queries. case (_: MicroBatchReadSupportProvider, _: StreamingWriteSupportProvider, t) diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java index adb269aa23..26d0f718f3 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java @@ -349,7 +349,7 @@ public class Column extends AbstractList { break; case FLOAT_TYPE: nulls.set(size, field == null); - doubleVars()[size] = field == null ? 0 : new Double(field.toString()); + doubleVars()[size] = field == null ? 0 : Double.valueOf(field.toString()); break; case DOUBLE_TYPE: nulls.set(size, field == null); diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d047953327..5823548a80 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -124,7 +124,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val lazyPruningEnabled = sparkSession.sqlContext.conf.manageFilesourcePartitions val tablePath = new Path(relation.tableMeta.location) - val fileFormat = fileFormatClass.newInstance() + val fileFormat = fileFormatClass.getConstructor().newInstance() val result = if (relation.isPartitioned) { val partitionSchema = relation.tableMeta.partitionSchema diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index 11afe1af32..c9fc3d4a02 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -217,7 +217,7 @@ private[hive] object HiveShim { instance.asInstanceOf[UDFType] } else { val func = Utils.getContextOrSparkClassLoader - .loadClass(functionClassName).newInstance.asInstanceOf[UDFType] + .loadClass(functionClassName).getConstructor().newInstance().asInstanceOf[UDFType] if (!func.isInstanceOf[UDF]) { // We cache the function if it's no the Simple UDF, // as we always have to create new instance for Simple UDF 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 9443fbb433..536bc4a3f4 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 @@ -132,7 +132,7 @@ class HadoopTableReader( val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHadoopConf.value.value - val deserializer = deserializerClass.newInstance() + val deserializer = deserializerClass.getConstructor().newInstance() deserializer.initialize(hconf, localTableDesc.getProperties) HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) } @@ -245,7 +245,7 @@ class HadoopTableReader( val localTableDesc = tableDesc createHadoopRdd(localTableDesc, inputPathStr, ifc).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val deserializer = localDeserializer.newInstance() + val deserializer = localDeserializer.getConstructor().newInstance() // SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema // information) may be defined in table properties. Here we should merge table properties // and partition properties before initializing the deserializer. Note that partition @@ -257,7 +257,7 @@ class HadoopTableReader( } deserializer.initialize(hconf, props) // get the table deserializer - val tableSerDe = localTableDesc.getDeserializerClass.newInstance() + val tableSerDe = localTableDesc.getDeserializerClass.getConstructor().newInstance() tableSerDe.initialize(hconf, localTableDesc.getProperties) // fill the non partition key attributes diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index bc9d4cd7f4..4d484904d2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -987,7 +987,7 @@ private[client] class Shim_v1_2 extends Shim_v1_1 { part: JList[String], deleteData: Boolean, purge: Boolean): Unit = { - val dropOptions = dropOptionsClass.newInstance().asInstanceOf[Object] + val dropOptions = dropOptionsClass.getConstructor().newInstance().asInstanceOf[Object] dropOptionsDeleteData.setBoolean(dropOptions, deleteData) dropOptionsPurge.setBoolean(dropOptions, purge) dropPartitionMethod.invoke(hive, dbName, tableName, part, dropOptions) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 4a7cd69019..d8d2a80e0e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -115,7 +115,8 @@ class HiveOutputWriter( private def tableDesc = fileSinkConf.getTableInfo private val serializer = { - val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] + val serializer = tableDesc.getDeserializerClass.getConstructor(). + newInstance().asInstanceOf[Serializer] serializer.initialize(jobConf, tableDesc.getProperties) serializer } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 92c6632ad7..fa940fe73b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -120,7 +120,7 @@ case class HiveTableScanExec( HiveShim.appendReadColumns(hiveConf, neededColumnIDs, output.map(_.name)) - val deserializer = tableDesc.getDeserializerClass.newInstance + val deserializer = tableDesc.getDeserializerClass.getConstructor().newInstance() deserializer.initialize(hiveConf, tableDesc.getProperties) // Specifies types and object inspectors of columns to be scanned. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala index 3328400b21..7b35a5f920 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala @@ -123,7 +123,7 @@ case class ScriptTransformationExec( var scriptOutputWritable: Writable = null val reusedWritableObject: Writable = if (null != outputSerde) { - outputSerde.getSerializedClass().newInstance + outputSerde.getSerializedClass().getConstructor().newInstance() } else { null } @@ -404,7 +404,8 @@ case class HiveScriptIOSchema ( columnTypes: Seq[DataType], serdeProps: Seq[(String, String)]): AbstractSerDe = { - val serde = Utils.classForName(serdeClassName).newInstance.asInstanceOf[AbstractSerDe] + val serde = Utils.classForName(serdeClassName).getConstructor(). + newInstance().asInstanceOf[AbstractSerDe] val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",") @@ -424,7 +425,8 @@ case class HiveScriptIOSchema ( inputStream: InputStream, conf: Configuration): Option[RecordReader] = { recordReaderClass.map { klass => - val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordReader] + val instance = Utils.classForName(klass).getConstructor(). + newInstance().asInstanceOf[RecordReader] val props = new Properties() // Can not use props.putAll(outputSerdeProps.toMap.asJava) in scala-2.12 // See https://github.com/scala/bug/issues/10418 @@ -436,7 +438,8 @@ case class HiveScriptIOSchema ( def recordWriter(outputStream: OutputStream, conf: Configuration): Option[RecordWriter] = { recordWriterClass.map { klass => - val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordWriter] + val instance = Utils.classForName(klass).getConstructor(). + newInstance().asInstanceOf[RecordWriter] instance.initialize(outputStream, conf) instance } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala index 0d4f040156..68a0c1213e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala @@ -152,7 +152,7 @@ class HiveParquetMetastoreSuite extends ParquetPartitioningTest { } (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").createOrReplaceTempView("jt") - (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a") + (1 to 10).map(i => Tuple1(Seq(Integer.valueOf(i), null))).toDF("a") .createOrReplaceTempView("jt_array") assert(spark.sqlContext.getConf(HiveUtils.CONVERT_METASTORE_PARQUET.key) == "true") 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 8d83dc8a8f..6f0b46b6a4 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 @@ -49,11 +49,11 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val clockClass = ssc.sc.conf.get( "spark.streaming.clock", "org.apache.spark.util.SystemClock") try { - Utils.classForName(clockClass).newInstance().asInstanceOf[Clock] + Utils.classForName(clockClass).getConstructor().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") - Utils.classForName(newClockClass).newInstance().asInstanceOf[Clock] + Utils.classForName(newClockClass).getConstructor().newInstance().asInstanceOf[Clock] } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 19b621f117..2332ee2ab9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -808,7 +808,8 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester // visible to mutableURLClassLoader val loader = new MutableURLClassLoader( Array(jar), appClassLoader) - assert(loader.loadClass("testClz").newInstance().toString == "testStringValue") + assert(loader.loadClass("testClz").getConstructor().newInstance().toString === + "testStringValue") // create and serialize Array[testClz] // scalastyle:off classforname