Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-45512][CORE][SQL][SS][DSTREAM] Fix compilation warnings related to other-nullary-override #43332

Closed
wants to merge 8 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}

Expand All @@ -133,7 +133,7 @@ class ExecutePlanResponseReattachableIterator(
}
}

override def hasNext(): Boolean = synchronized {
override def hasNext: Boolean = synchronized {
if (resultComplete) {
// After response complete response
return false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ private[sql] class GrpcRetryHandler(
}
}

override def next: U = {
override def next(): U = {
retryIter(_.next)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/TaskContextImpl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/org/apache/spark/ShuffleSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}
Expand Down
7 changes: 0 additions & 7 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -2964,21 +2964,14 @@
<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.
-->
<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`
Expand Down
4 changes: 0 additions & 4 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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)
}

/**
Expand Down Expand Up @@ -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)
}

/**
Expand Down Expand Up @@ -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)
}
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand All @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ class WholeStageCodegenEvaluatorFactory(
if (!v) durationMs += buffer.durationMs()
v
}
override def next: InternalRow = buffer.next()
override def next(): InternalRow = buffer.next()
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ case class BroadcastNestedLoopJoinExec(
}
}

override def hasNext(): Boolean = {
override def hasNext: Boolean = {
resultRow != null || findNextMatch()
}
override def next(): InternalRow = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down Expand Up @@ -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)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down