Skip to content
Closed
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 @@ -2965,21 +2965,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
Loading