This is an automated email from the ASF dual-hosted git repository.
srowen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 3b46cc81614 [SPARK-45512][CORE][SQL][SS][DSTREAM] Fix compilation
warnings related to `other-nullary-override`
3b46cc81614 is described below
commit 3b46cc816143d5bb553e86e8b716c28982cb5748
Author: YangJie <[email protected]>
AuthorDate: Tue Oct 17 07:34:06 2023 -0500
[SPARK-45512][CORE][SQL][SS][DSTREAM] Fix compilation warnings related to
`other-nullary-override`
### What changes were proposed in this pull request?
This PR fixes two compilation warnings related to `other-nullary-override`
```
[error]
/Users/yangjie01/SourceCode/git/spark-mine-sbt/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala:36:16:
method with a single empty parameter list overrides method hasNext in trait
Iterator defined without a parameter list [quickfixable]
[error] Applicable -Wconf / nowarn filters for this fatal warning:
msg=<part of the message>, cat=other-nullary-override,
site=org.apache.spark.sql.connect.client.WrappedCloseableIterator
[error] override def hasNext(): Boolean = innerIterator.hasNext
[error] ^
[error]
/Users/yangjie01/SourceCode/git/spark-mine-sbt/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala:136:16:
method without a parameter list overrides method hasNext in class
WrappedCloseableIterator defined with a single empty parameter list
[quickfixable]
[error] Applicable -Wconf / nowarn filters for this fatal warning:
msg=<part of the message>, cat=other-nullary-override,
site=org.apache.spark.sql.connect.client.ExecutePlanResponseReattachableIterator
[error] override def hasNext: Boolean = synchronized {
[error] ^
[error]
/Users/yangjie01/SourceCode/git/spark-mine-sbt/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala:73:20:
method without a parameter list overrides method hasNext in class
WrappedCloseableIterator defined with a single empty parameter list
[quickfixable]
[error] Applicable -Wconf / nowarn filters for this fatal warning:
msg=<part of the message>, cat=other-nullary-override,
site=org.apache.spark.sql.connect.client.GrpcExceptionConverter.convertIterator
[error] override def hasNext: Boolean = {
[error] ^
[error]
/Users/yangjie01/SourceCode/git/spark-mine-sbt/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala:77:18:
method without a parameter list overrides method next in class
WrappedCloseableIterator defined with a single empty parameter list
[quickfixable]
[error] Applicable -Wconf / nowarn filters for this fatal warning:
msg=<part of the message>, cat=other-nullary-override,
site=org.apache.spark.sql.connect.client.GrpcRetryHandler.RetryIterator
[error] override def next: U = {
[error] ^
[error]
/Users/yangjie01/SourceCode/git/spark-mine-sbt/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala:81:18:
method without a parameter list overrides method hasNext in class
WrappedCloseableIterator defined with a single empty parameter list
[quickfixable]
[error] Applicable -Wconf / nowarn filters for this fatal warning:
msg=<part of the message>, cat=other-nullary-override,
site=org.apache.spark.sql.connect.client.GrpcRetryHandler.RetryIterator
[error] override def hasNext: Boolean = {
[error]
```
and removes the corresponding suppression rules from the compilation options
```
"-Wconf:cat=other-nullary-override:wv",
```
On the other hand, the code corresponding to the following three
suppression rules no longer exists, so the corresponding suppression rules were
also cleaned up in this pr.
```
"-Wconf:cat=lint-multiarg-infix:wv",
"-Wconf:msg=method with a single empty parameter list overrides method
without any parameter list:s",
"-Wconf:msg=method without a parameter list overrides a method with a
single empty one:s",
```
### Why are the changes needed?
Code clean up.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass GitHub Actions
### Was this patch authored or co-authored using generative AI tooling?
No
Closes #43332 from LuciferYang/other-nullary-override.
Lead-authored-by: YangJie <[email protected]>
Co-authored-by: yangjie01 <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
---
.../org/apache/spark/sql/avro/AvroRowReaderSuite.scala | 10 +++++-----
.../spark/sql/connect/client/CloseableIterator.scala | 2 +-
.../ExecutePlanResponseReattachableIterator.scala | 4 ++--
.../spark/sql/connect/client/GrpcRetryHandler.scala | 2 +-
.../sql/kafka010/KafkaDontFailOnDataLossSuite.scala | 2 +-
.../spark/streaming/kafka010/KafkaRDDSuite.scala | 4 ++--
.../main/scala/org/apache/spark/TaskContextImpl.scala | 2 +-
.../spark/deploy/history/FsHistoryProvider.scala | 2 +-
.../src/main/scala/org/apache/spark/rdd/PipedRDD.scala | 4 ++--
.../org/apache/spark/scheduler/TaskSetManager.scala | 2 +-
.../org/apache/spark/storage/memory/MemoryStore.scala | 2 +-
.../scala/org/apache/spark/util/HadoopFSUtils.scala | 4 ++--
.../spark/util/collection/ExternalAppendOnlyMap.scala | 2 +-
.../apache/spark/util/collection/ExternalSorter.scala | 2 +-
.../src/test/scala/org/apache/spark/ShuffleSuite.scala | 2 +-
pom.xml | 7 -------
project/SparkBuild.scala | 4 ----
.../sql/catalyst/expressions/DynamicPruning.scala | 2 +-
.../FunctionTableSubqueryArgumentExpression.scala | 2 +-
.../catalyst/expressions/higherOrderFunctions.scala | 2 +-
.../spark/sql/catalyst/expressions/subquery.scala | 8 ++++----
.../execution/ExternalAppendOnlyUnsafeRowArray.scala | 4 ++--
.../execution/WholeStageCodegenEvaluatorFactory.scala | 2 +-
.../aggregate/ObjectAggregationIterator.scala | 2 +-
.../execution/joins/BroadcastNestedLoopJoinExec.scala | 2 +-
.../apache/spark/sql/execution/metric/SQLMetrics.scala | 2 +-
.../streaming/state/HDFSBackedStateStoreMap.scala | 4 ++--
.../org/apache/spark/sql/execution/subquery.scala | 2 +-
.../scala/org/apache/spark/sql/SSBQuerySuite.scala | 2 +-
.../apache/spark/sql/SparkSessionExtensionSuite.scala | 4 ++--
.../sql/execution/command/v1/DropNamespaceSuite.scala | 2 +-
.../spark/sql/execution/metric/SQLMetricsSuite.scala | 18 +++++++++---------
.../sql/execution/ui/AllExecutionsPageSuite.scala | 2 +-
.../spark/sql/hive/execution/Hive_2_1_DDLSuite.scala | 2 +-
.../hive/execution/PruneHiveTablePartitionsSuite.scala | 2 +-
.../org/apache/spark/streaming/InputStreamsSuite.scala | 2 +-
36 files changed, 56 insertions(+), 67 deletions(-)
diff --git
a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala
b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala
index 70d0bc6c0ad..1c450d4f166 100644
---
a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala
+++
b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala
@@ -80,19 +80,19 @@ class AvroRowReaderSuite
override def hasNext: Boolean = hasNextRow
- override def next: InternalRow = nextRow
+ override def next(): InternalRow = nextRow
}
assert(it.hasNext == true)
- assert(it.next.getInt(0) == 1)
+ assert(it.next().getInt(0) == 1)
// test no intervening next
assert(it.hasNext == true)
assert(it.hasNext == true)
// test no intervening hasNext
- assert(it.next.getInt(0) == 2)
- assert(it.next.getInt(0) == 3)
+ assert(it.next().getInt(0) == 2)
+ assert(it.next().getInt(0) == 3)
assert(it.hasNext == false)
assertThrows[NoSuchElementException] {
- it.next
+ it.next()
}
}
}
diff --git
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala
index 810158b2ac8..4ec6828d885 100644
---
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala
+++
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala
@@ -33,7 +33,7 @@ private[sql] abstract class WrappedCloseableIterator[E]
extends CloseableIterato
override def next(): E = innerIterator.next()
- override def hasNext(): Boolean = innerIterator.hasNext
+ override def hasNext: Boolean = innerIterator.hasNext
override def close(): Unit = innerIterator match {
case it: CloseableIterator[E] => it.close()
diff --git
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala
index 4c0c1d1f390..9d134f59354 100644
---
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala
+++
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala
@@ -108,7 +108,7 @@ class ExecutePlanResponseReattachableIterator(
override def next(): proto.ExecutePlanResponse = synchronized {
// hasNext will trigger reattach in case the stream completed without
resultComplete
- if (!hasNext()) {
+ if (!hasNext) {
throw new java.util.NoSuchElementException()
}
@@ -133,7 +133,7 @@ class ExecutePlanResponseReattachableIterator(
}
}
- override def hasNext(): Boolean = synchronized {
+ override def hasNext: Boolean = synchronized {
if (resultComplete) {
// After response complete response
return false
diff --git
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala
index 3c0b750fd46..74c8423ce43 100644
---
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala
+++
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala
@@ -74,7 +74,7 @@ private[sql] class GrpcRetryHandler(
}
}
- override def next: U = {
+ override def next(): U = {
retryIter(_.next)
}
diff --git
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
index 54ce1717acc..2e0690f2288 100644
---
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
+++
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
@@ -47,7 +47,7 @@ trait KafkaMissingOffsetsTest extends SharedSparkSession {
protected var testUtils: KafkaTestUtils = _
- override def createSparkSession(): TestSparkSession = {
+ override def createSparkSession: TestSparkSession = {
// Set maxRetries to 3 to handle NPE from `poll` when deleting a topic
new TestSparkSession(new SparkContext("local[2,3]", "test-sql-context",
sparkConf))
}
diff --git
a/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala
b/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala
index 735ec2f7b44..986943ac6a1 100644
---
a/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala
+++
b/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala
@@ -48,14 +48,14 @@ class KafkaRDDSuite extends SparkFunSuite {
private var sc: SparkContext = _
- override def beforeAll: Unit = {
+ override def beforeAll(): Unit = {
super.beforeAll()
sc = new SparkContext(sparkConf)
kafkaTestUtils = new KafkaTestUtils
kafkaTestUtils.setup()
}
- override def afterAll: Unit = {
+ override def afterAll(): Unit = {
try {
try {
if (sc != null) {
diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
index 4cc5f165794..8d2c2ab9bc4 100644
--- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
@@ -292,5 +292,5 @@ private[spark] class TaskContextImpl(
private[spark] override def fetchFailed: Option[FetchFailedException] =
_fetchFailedException
- private[spark] override def getLocalProperties(): Properties =
localProperties
+ private[spark] override def getLocalProperties: Properties = localProperties
}
diff --git
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index abb65c7311e..b5afa86180b 100644
---
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -1288,7 +1288,7 @@ private[history] class FsHistoryProvider(conf: SparkConf,
clock: Clock)
val diskStore = KVUtils.open(lease.tmpPath, metadata, conf, live = false)
hybridStore.setDiskStore(diskStore)
hybridStore.switchToDiskStore(new HybridStore.SwitchToDiskStoreListener {
- override def onSwitchToDiskStoreSuccess: Unit = {
+ override def onSwitchToDiskStoreSuccess(): Unit = {
logInfo(s"Completely switched to diskStore for app $appId /
${attempt.info.attemptId}.")
diskStore.close()
val newStorePath = lease.commit(appId, attempt.info.attemptId)
diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
index 0359f6c6d38..4f3683ef2df 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -185,13 +185,13 @@ private[spark] class PipedRDD[T: ClassTag](
val lines = Source.fromInputStream(proc.getInputStream)(encoding).getLines
new Iterator[String] {
def next(): String = {
- if (!hasNext()) {
+ if (!hasNext) {
throw SparkCoreErrors.noSuchElementError()
}
lines.next()
}
- def hasNext(): Boolean = {
+ def hasNext: Boolean = {
val result = if (lines.hasNext) {
true
} else {
diff --git
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 4d5f32ac67b..de9c58e8d90 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -1032,7 +1032,7 @@ private[spark] class TaskSetManager(
override def removeSchedulable(schedulable: Schedulable): Unit = {}
- override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
+ override def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] = {
val sortedTaskSetQueue = new ArrayBuffer[TaskSetManager]()
sortedTaskSetQueue += this
sortedTaskSetQueue
diff --git
a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
index dafa67b091d..88bb3eb7105 100644
--- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
@@ -746,7 +746,7 @@ private class SerializedValuesHolder[T](
// We successfully unrolled the entirety of this block
serializationStream.close()
- override def preciseSize(): Long = bbos.size
+ override def preciseSize: Long = bbos.size
override def build(): MemoryEntry[T] =
SerializedMemoryEntry[T](bbos.toChunkedByteBuffer, memoryMode, classTag)
diff --git a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
index a033b817f5a..5cd93dfae35 100644
--- a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
@@ -91,7 +91,7 @@ private[spark] object HadoopFSUtils extends Logging {
val remoteIter = path.getFileSystem(hadoopConf).listFiles(path, true)
val statues = new Iterator[LocatedFileStatus]() {
def next(): LocatedFileStatus = remoteIter.next
- def hasNext(): Boolean = remoteIter.hasNext
+ def hasNext: Boolean = remoteIter.hasNext
}.filterNot(status =>
shouldFilterOutPath(status.getPath.toString.substring(prefixLength)))
.filter(f => filter.accept(f.getPath))
.toArray
@@ -209,7 +209,7 @@ private[spark] object HadoopFSUtils extends Logging {
val remoteIter = fs.listLocatedStatus(path)
new Iterator[LocatedFileStatus]() {
def next(): LocatedFileStatus = remoteIter.next
- def hasNext(): Boolean = remoteIter.hasNext
+ def hasNext: Boolean = remoteIter.hasNext
}.toArray
case _ => fs.listStatus(path)
}
diff --git
a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 93efaafa43b..71f3b0b4b4d 100644
---
a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++
b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -592,7 +592,7 @@ class ExternalAppendOnlyMap[K, V, C](
}
}
- override def hasNext(): Boolean = cur != null
+ override def hasNext: Boolean = cur != null
override def next(): (K, C) = {
val r = cur
diff --git
a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index a68e0de70c5..42501729e1b 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -842,7 +842,7 @@ private[spark] class ExternalSorter[K, V, C](
}
}
- override def hasNext(): Boolean = cur != null
+ override def hasNext: Boolean = cur != null
override def next(): ((Int, K), C) = {
val r = cur
diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
index c1b9af37ce7..0b6fce57727 100644
--- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
@@ -502,7 +502,7 @@ class InterleaveIterators[T, R](
class BarrierIterator[E](id: Int, sub: Iterator[E]) extends Iterator[E] {
def hasNext: Boolean = sub.hasNext
- def next: E = {
+ def next(): E = {
barrier.await()
sub.next()
}
diff --git a/pom.xml b/pom.xml
index b6804dfb75f..824ae49f6da 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2965,12 +2965,7 @@
<arg>-target:17</arg>
<arg>-Wconf:cat=deprecation:wv,any:e</arg>
<arg>-Wunused:imports</arg>
- <!--
- TODO(SPARK-33805): Undo the corresponding deprecated usage
suppression rule after fixed
- -->
<arg>-Wconf:cat=scaladoc:wv</arg>
- <arg>-Wconf:cat=lint-multiarg-infix:wv</arg>
- <arg>-Wconf:cat=other-nullary-override:wv</arg>
<!--
SPARK-33775 Suppress compilation warnings that contain the
following contents.
TODO(SPARK-33805): Undo the corresponding deprecated usage
suppression rule after fixed.
@@ -2978,8 +2973,6 @@
<arg>-Wconf:msg=^(?=.*?method|value|type|object|trait|inheritance)(?=.*?deprecated)(?=.*?since
2.13).+$:s</arg>
<arg>-Wconf:msg=^(?=.*?Widening conversion from)(?=.*?is
deprecated because it loses precision).+$:s</arg>
<arg>-Wconf:msg=Auto-application to \`\(\)\` is
deprecated:s</arg>
- <arg>-Wconf:msg=method with a single empty parameter list
overrides method without any parameter list:s</arg>
- <arg>-Wconf:msg=method without a parameter list overrides a
method with a single empty one:s</arg>
<!--
SPARK-35574 Prevent the recurrence of compilation warnings
related to
`procedure syntax is deprecated`
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index f433fbc78a2..d2c9771b0f8 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -232,16 +232,12 @@ object SparkBuild extends PomBuild {
"-Wconf:cat=deprecation:wv,any:e",
// 2.13-specific warning hits to be muted (as narrowly as possible)
and addressed separately
"-Wunused:imports",
- "-Wconf:cat=lint-multiarg-infix:wv",
- "-Wconf:cat=other-nullary-override:wv",
// SPARK-33775 Suppress compilation warnings that contain the
following contents.
// TODO(SPARK-33805): Undo the corresponding deprecated usage
suppression rule after
// fixed.
"-Wconf:msg=^(?=.*?method|value|type|object|trait|inheritance)(?=.*?deprecated)(?=.*?since
2.13).+$:s",
"-Wconf:msg=^(?=.*?Widening conversion from)(?=.*?is deprecated
because it loses precision).+$:s",
"-Wconf:msg=Auto-application to \\`\\(\\)\\` is deprecated:s",
- "-Wconf:msg=method with a single empty parameter list overrides method
without any parameter list:s",
- "-Wconf:msg=method without a parameter list overrides a method with a
single empty one:s",
// SPARK-35574 Prevent the recurrence of compilation warnings related
to `procedure syntax is deprecated`
"-Wconf:cat=deprecation&msg=procedure syntax is deprecated:e",
// SPARK-35496 Upgrade Scala to 2.13.7 and suppress:
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala
index 1e94188bd18..ec6925eaa98 100644
---
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala
@@ -73,7 +73,7 @@ case class DynamicPruningSubquery(
pruningKey.dataType == buildKeys(broadcastKeyIndex).dataType
}
- final override def nodePatternsInternal: Seq[TreePattern] =
Seq(DYNAMIC_PRUNING_SUBQUERY)
+ final override def nodePatternsInternal(): Seq[TreePattern] =
Seq(DYNAMIC_PRUNING_SUBQUERY)
override def toString: String = s"dynamicpruning#${exprId.id}
$conditionString"
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala
index 209eb02e3b3..da2dd9b1e25 100644
---
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala
@@ -93,7 +93,7 @@ case class FunctionTableSubqueryArgumentExpression(
newChildren: IndexedSeq[Expression]):
FunctionTableSubqueryArgumentExpression =
copy(outerAttrs = newChildren)
- final override def nodePatternsInternal: Seq[TreePattern] =
+ final override def nodePatternsInternal(): Seq[TreePattern] =
Seq(FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION)
def hasRepartitioning: Boolean = withSinglePartition ||
partitionByExpressions.nonEmpty
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
index fec1df108bc..4df6a5e7666 100644
---
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
@@ -248,7 +248,7 @@ trait SimpleHigherOrderFunction extends HigherOrderFunction
with BinaryLike[Expr
def argumentType: AbstractDataType
- override def argumentTypes(): Seq[AbstractDataType] = argumentType :: Nil
+ override def argumentTypes: Seq[AbstractDataType] = argumentType :: Nil
def function: Expression
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
index bbcd9e336b4..9d8b5de271e 100644
---
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
@@ -295,7 +295,7 @@ case class ScalarSubquery(
outerAttrs = newChildren.take(outerAttrs.size),
joinCond = newChildren.drop(outerAttrs.size))
- final override def nodePatternsInternal: Seq[TreePattern] =
Seq(SCALAR_SUBQUERY)
+ final override def nodePatternsInternal(): Seq[TreePattern] =
Seq(SCALAR_SUBQUERY)
}
object ScalarSubquery {
@@ -339,7 +339,7 @@ case class LateralSubquery(
outerAttrs = newChildren.take(outerAttrs.size),
joinCond = newChildren.drop(outerAttrs.size))
- final override def nodePatternsInternal: Seq[TreePattern] =
Seq(LATERAL_SUBQUERY)
+ final override def nodePatternsInternal(): Seq[TreePattern] =
Seq(LATERAL_SUBQUERY)
}
/**
@@ -397,7 +397,7 @@ case class ListQuery(
outerAttrs = newChildren.take(outerAttrs.size),
joinCond = newChildren.drop(outerAttrs.size))
- final override def nodePatternsInternal: Seq[TreePattern] =
Seq(LIST_SUBQUERY)
+ final override def nodePatternsInternal(): Seq[TreePattern] =
Seq(LIST_SUBQUERY)
}
/**
@@ -452,5 +452,5 @@ case class Exists(
outerAttrs = newChildren.take(outerAttrs.size),
joinCond = newChildren.drop(outerAttrs.size))
- final override def nodePatternsInternal: Seq[TreePattern] =
Seq(EXISTS_SUBQUERY)
+ final override def nodePatternsInternal(): Seq[TreePattern] =
Seq(EXISTS_SUBQUERY)
}
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
index 4147d75186d..56289d73c07 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
@@ -209,7 +209,7 @@ private[sql] class ExternalAppendOnlyUnsafeRowArray(
private var currentIndex = startIndex
- override def hasNext(): Boolean = !isModified() && currentIndex < numRows
+ override def hasNext: Boolean = !isModified() && currentIndex < numRows
override def next(): UnsafeRow = {
throwExceptionIfModified()
@@ -226,7 +226,7 @@ private[sql] class ExternalAppendOnlyUnsafeRowArray(
private val currentRow = new UnsafeRow(numFieldPerRow)
- override def hasNext(): Boolean = !isModified() && iterator.hasNext
+ override def hasNext: Boolean = !isModified() && iterator.hasNext
override def next(): UnsafeRow = {
throwExceptionIfModified()
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenEvaluatorFactory.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenEvaluatorFactory.scala
index edf5774cf02..f0f9c7cbc95 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenEvaluatorFactory.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenEvaluatorFactory.scala
@@ -44,7 +44,7 @@ class WholeStageCodegenEvaluatorFactory(
if (!v) durationMs += buffer.durationMs()
v
}
- override def next: InternalRow = buffer.next()
+ override def next(): InternalRow = buffer.next()
}
}
}
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala
index fac3f7d6d8a..6d05be72b36 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala
@@ -254,7 +254,7 @@ class SortBasedAggregator(
private var result: AggregationBufferEntry = _
private var groupingKey: UnsafeRow = _
- override def hasNext(): Boolean = {
+ override def hasNext: Boolean = {
result != null || findNextSortedGroup()
}
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
index c56cc294a9f..32d23136225 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
@@ -181,7 +181,7 @@ case class BroadcastNestedLoopJoinExec(
}
}
- override def hasNext(): Boolean = {
+ override def hasNext: Boolean = {
resultRow != null || findNextMatch()
}
override def next(): InternalRow = {
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
index 3326c5d4cb9..69d288ae75c 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
@@ -63,7 +63,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L)
extends Accumulato
this.getClass.getName, other.getClass.getName)
}
- override def isZero(): Boolean = _value == _zeroValue
+ override def isZero: Boolean = _value == _zeroValue
override def add(v: Long): Unit = {
if (_value < 0) _value = 0
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreMap.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreMap.scala
index a79a856b0ec..59b01b5b454 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreMap.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreMap.scala
@@ -54,7 +54,7 @@ object HDFSBackedStateStoreMap {
class NoPrefixHDFSBackedStateStoreMap extends HDFSBackedStateStoreMap {
private val map = new HDFSBackedStateStoreMap.MapType()
- override def size: Int = map.size()
+ override def size(): Int = map.size()
override def get(key: UnsafeRow): UnsafeRow = map.get(key)
@@ -103,7 +103,7 @@ class PrefixScannableHDFSBackedStateStoreMap(
UnsafeProjection.create(refs)
}
- override def size: Int = map.size()
+ override def size(): Int = map.size()
override def get(key: UnsafeRow): UnsafeRow = map.get(key)
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
index 41230c7792c..58f720154df 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
@@ -127,7 +127,7 @@ case class InSubqueryExec(
override def nullable: Boolean = child.nullable
override def toString: String = s"$child IN ${plan.name}"
override def withNewPlan(plan: BaseSubqueryExec): InSubqueryExec = copy(plan
= plan)
- final override def nodePatternsInternal: Seq[TreePattern] =
Seq(IN_SUBQUERY_EXEC)
+ final override def nodePatternsInternal(): Seq[TreePattern] =
Seq(IN_SUBQUERY_EXEC)
def updateResult(): Unit = {
val rows = plan.executeCollect()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala
index 099b559105f..8df91c6dbda 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.resourceToString
*/
class SSBQuerySuite extends BenchmarkQueryTest {
- override def beforeAll: Unit = {
+ override def beforeAll(): Unit = {
super.beforeAll
sql(
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
index 21518085ca4..c1b5d2761f7 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
@@ -824,7 +824,7 @@ class ColumnarWriteExec(
staticPartitions: TablePartitionSpec) extends WriteFilesExec(
child, fileFormat, partitionColumns, bucketSpec, options, staticPartitions) {
- override def supportsColumnar(): Boolean = true
+ override def supportsColumnar: Boolean = true
override def doExecuteWrite(writeFilesSpec: WriteFilesSpec):
RDD[WriterCommitMessage] = {
assert(child.supportsColumnar)
@@ -846,7 +846,7 @@ class BrokenColumnarAdd(
failOnError: Boolean = false)
extends Add(left, right, EvalMode.fromBoolean(failOnError)) with
ColumnarExpression {
- override def supportsColumnar(): Boolean = left.supportsColumnar &&
right.supportsColumnar
+ override def supportsColumnar: Boolean = left.supportsColumnar &&
right.supportsColumnar
override def columnarEval(batch: ColumnarBatch): Any = {
var lhs: Any = null
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala
index ea03654f921..647247cc833 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala
@@ -32,7 +32,7 @@ trait DropNamespaceSuiteBase extends
command.DropNamespaceSuiteBase
with command.TestsV1AndV2Commands {
override protected def builtinTopNamespaces: Seq[String] = Seq("default")
- override protected def namespaceAlias(): String = "database"
+ override protected def namespaceAlias: String = "database"
test("drop default namespace") {
checkError(
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
index 6347757e178..e71451f2f74 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
@@ -940,25 +940,25 @@ class SQLMetricsSuite extends SharedSparkSession with
SQLMetricsTestUtils
assert(SQLMetrics.createSizeMetric(sparkContext, name = "m", initValue =
-1).value === 0)
assert(SQLMetrics.createSizeMetric(sparkContext, name = "m", initValue =
5).value === 5)
- assert(SQLMetrics.createSizeMetric(sparkContext, name = "m").isZero())
- assert(SQLMetrics.createSizeMetric(sparkContext, name = "m", initValue =
-1).isZero())
- assert(SQLMetrics.createSizeMetric(sparkContext, name = "m", initValue =
5).isZero())
+ assert(SQLMetrics.createSizeMetric(sparkContext, name = "m").isZero)
+ assert(SQLMetrics.createSizeMetric(sparkContext, name = "m", initValue =
-1).isZero)
+ assert(SQLMetrics.createSizeMetric(sparkContext, name = "m", initValue =
5).isZero)
assert(SQLMetrics.createTimingMetric(sparkContext, name = "m").value === 0)
assert(SQLMetrics.createTimingMetric(sparkContext, name = "m", initValue =
-1).value === 0)
assert(SQLMetrics.createTimingMetric(sparkContext, name = "m", initValue =
5).value === 5)
- assert(SQLMetrics.createTimingMetric(sparkContext, name = "m").isZero())
- assert(SQLMetrics.createTimingMetric(sparkContext, name = "m", initValue =
-1).isZero())
- assert(SQLMetrics.createTimingMetric(sparkContext, name = "m", initValue =
5).isZero())
+ assert(SQLMetrics.createTimingMetric(sparkContext, name = "m").isZero)
+ assert(SQLMetrics.createTimingMetric(sparkContext, name = "m", initValue =
-1).isZero)
+ assert(SQLMetrics.createTimingMetric(sparkContext, name = "m", initValue =
5).isZero)
assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m").value
=== 0)
assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m",
initValue = -1).value === 0)
assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m",
initValue = 5).value === 5)
- assert(SQLMetrics.createNanoTimingMetric(sparkContext, name =
"m").isZero())
- assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m",
initValue = -1).isZero())
- assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m",
initValue = 5).isZero())
+ assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m").isZero)
+ assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m",
initValue = -1).isZero)
+ assert(SQLMetrics.createNanoTimingMetric(sparkContext, name = "m",
initValue = 5).isZero)
}
}
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala
index fe5e99d3f3f..17a5e0daaaf 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala
@@ -275,7 +275,7 @@ class AllExecutionsPageWithInMemoryStoreSuite extends
AllExecutionsPageSuite {
@SlowSQLTest
class AllExecutionsPageWithRocksDBBackendSuite extends AllExecutionsPageSuite {
private val storePath = Utils.createTempDir()
- override protected def createStatusStore(): SQLAppStatusStore = {
+ override protected def createStatusStore: SQLAppStatusStore = {
val conf = sparkContext.conf
conf.set(LIVE_UI_LOCAL_STORE_DIR, storePath.getCanonicalPath)
val appStatusStore = AppStatusStore.createLiveStore(conf)
diff --git
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala
index 60a02461c93..f9dbae9b1aa 100644
---
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala
+++
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala
@@ -62,7 +62,7 @@ class Hive_2_1_DDLSuite extends SparkFunSuite with
TestHiveSingleton {
new HiveExternalCatalog(sparkConf, hadoopConf)
}
- override def afterEach: Unit = {
+ override def afterEach(): Unit = {
catalog.listTables("default").foreach { t =>
catalog.dropTable("default", t, true, false)
}
diff --git
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala
index 42601be08e1..9a512871b54 100644
---
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala
+++
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala
@@ -34,7 +34,7 @@ import org.apache.spark.tags.SlowHiveTest
@SlowHiveTest
class PruneHiveTablePartitionsSuite extends PrunePartitionSuiteBase with
TestHiveSingleton {
- override def format(): String = "hive"
+ override def format: String = "hive"
object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
diff --git
a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index e00651fe910..2a2243316a4 100644
---
a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++
b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -457,7 +457,7 @@ class InputStreamsSuite extends TestSuiteBase with
BeforeAndAfter {
}
class TestReceiverInputDStream extends ReceiverInputDStream[String](ssc)
{
- def getReceiver: Receiver[String] = null
+ def getReceiver(): Receiver[String] = null
}
// Register input streams
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]