[SPARK-35526][CORE][SQL][ML][MLLIB] Re-Cleanup procedure syntax is deprecated
compilation warning in Scala 2.13
### What changes were proposed in this pull request? After SPARK-29291 and SPARK-33352, there are still some compilation warnings about `procedure syntax is deprecated` as follows: ``` [WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:723: [deprecation | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `registerMergeResult`'s return type [WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:748: [deprecation | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `unregisterMergeResult`'s return type [WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala:223: [deprecation | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `testSimpleSpillingForAllCodecs`'s return type [WARNING] [Warn] /spark/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala:53: [deprecation | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `runBLASBenchmark`'s return type [WARNING] [Warn] /spark/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala:110: [deprecation | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `assertEmptyRootPath`'s return type [WARNING] [Warn] /spark/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:602: [deprecation | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `executeCTASWithNonEmptyLocation`'s return type ``` So the main change of this pr is cleanup these compilation warnings. ### Why are the changes needed? Eliminate compilation warnings in Scala 2.13 and this change should be compatible with Scala 2.12 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass the Jenkins or GitHub Action Closes #32669 from LuciferYang/re-clean-procedure-syntax. Authored-by: yangjie01 <yangjie01@baidu.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
This commit is contained in:
parent
16d9de815e
commit
09d039da56
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in a new issue