diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index b749d7e862..ea9e641101 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -720,7 +720,7 @@ private[spark] class MapOutputTrackerMaster( } } - def registerMergeResult(shuffleId: Int, reduceId: Int, status: MergeStatus) { + def registerMergeResult(shuffleId: Int, reduceId: Int, status: MergeStatus): Unit = { shuffleStatuses(shuffleId).addMergeResult(reduceId, status) } @@ -745,7 +745,7 @@ private[spark] class MapOutputTrackerMaster( shuffleId: Int, reduceId: Int, bmAddress: BlockManagerId, - mapId: Option[Int] = None) { + mapId: Option[Int] = None): Unit = { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => val mergeStatus = shuffleStatus.mergeStatuses(reduceId) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 81a145906d..3f6cb2475a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -220,7 +220,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite testSimpleSpilling() } - private def testSimpleSpillingForAllCodecs(encrypt: Boolean) { + private def testSimpleSpillingForAllCodecs(encrypt: Boolean): Unit = { // Keep track of which compression codec we're using to report in test failure messages var lastCompressionCodec: Option[String] = None try { diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala index 86a576826b..699d841e06 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala @@ -50,7 +50,7 @@ object BLASBenchmark extends BenchmarkBase { println("nativeBLAS = " + nativeBLAS.getClass.getName) // scalastyle:on println - def runBLASBenchmark(name: String, n: Int)(bench: NetlibBLAS => Unit) { + def runBLASBenchmark(name: String, n: Int)(bench: NetlibBLAS => Unit): Unit = { val benchmark = new Benchmark(name, n, iters, warmupTime = 30.seconds, minTime = 30.seconds, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala index ca71395fc5..ca51b88de0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala @@ -107,7 +107,7 @@ object DataWritingCommand { * @param saveMode Save mode of the table. * @param hadoopConf Configuration. */ - def assertEmptyRootPath(tablePath: URI, saveMode: SaveMode, hadoopConf: Configuration) { + def assertEmptyRootPath(tablePath: URI, saveMode: SaveMode, hadoopConf: Configuration): Unit = { if (saveMode == SaveMode.ErrorIfExists && !SQLConf.get.allowNonEmptyLocationInCTAS) { val filePath = new org.apache.hadoop.fs.Path(tablePath) val fs = filePath.getFileSystem(hadoopConf) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 462d3f3423..684b2f22ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -599,7 +599,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } test("SPARK-28551: CTAS Hive Table should be with non-existent or empty location") { - def executeCTASWithNonEmptyLocation(tempLocation: String) { + def executeCTASWithNonEmptyLocation(tempLocation: String): Unit = { sql(s"CREATE TABLE ctas1(id string) stored as rcfile LOCATION '$tempLocation/ctas1'") sql("INSERT INTO TABLE ctas1 SELECT 'A' ") sql(s"""CREATE TABLE ctas_with_existing_location stored as rcfile LOCATION