[SPARK-45610][BUILD][CORE][SQL][SS][CONNECT][GRAPHX][DSTREAM][ML][MLLIB][K8S][YARN][SHELL][PYTHON][R][AVRO][UI][EXAMPLES] Fix the compilation warning "Auto-application to `()` is deprecated" and turn it into a compilation error
### What changes were proposed in this pull request?
This PR mainly does two things:
1. Clean up all compilation warnings related to "Auto-application to () is deprecated".
2. Change the compilation options to convert this compilation warning into a compilation error.
Additionally, due to an issue with scalatest(https://github.com/scalatest/scalatest/issues/2297), there are some false positives. Therefore, this PR has added the corresponding rules to suppress them, and left the corresponding TODO(SPARK-45615). We can clean up these rules after scalatest fixes this issue(https://github.com/scalatest/scalatest/pull/2298).
### Why are the changes needed?
1. Clean up the deprecated usage methods.
2. As this compilation warning will become a compilation error in Scala 3, to ensure it does not occur again, this PR also converts it into a compilation error in Scala 2.13.
For example, for the following code:
```scala
class Foo {
def isEmpty(): Boolean = true
}
val foo = new Foo
val ret = foo.isEmpty
```
In Scala 2.13:
```
Welcome to Scala 2.13.12 (OpenJDK 64-Bit Server VM, Java 17.0.8).
Type in expressions for evaluation. Or try :help.
scala> class Foo {
| def isEmpty(): Boolean = true
| }
class Foo
scala> val foo = new Foo
|
val foo: Foo = Foo7e15f4d4
scala> val ret = foo.isEmpty
^
warning: Auto-application to `()` is deprecated. Supply the empty argument list `()` explicitly to invoke method isEmpty,
or remove the empty argument list from its definition (Java-defined methods are exempt).
In Scala 3, an unapplied method like this will be eta-expanded into a function. [quickfixable]
val ret: Boolean = true
```
In Scala 3:
```
Welcome to Scala 3.3.1 (17.0.8, Java OpenJDK 64-Bit Server VM).
Type in expressions for evaluation. Or try :help.
scala> class Foo {
| def isEmpty(): Boolean = true
| }
// defined class Foo
scala> val foo = new Foo
val foo: Foo = Foo150d6eaf
scala> val ret = foo.isEmpty
-- [E100] Syntax Error: --------------------------------------------------------
1 |val ret = foo.isEmpty
| ^^^^^^^^^^^
| method isEmpty in class Foo must be called with () argument
|-----------------------------------------------------------------------------
| Explanation (enabled by `-explain`)
|- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
| Previously an empty argument list () was implicitly inserted when calling a nullary method without arguments. E.g.
|
| def next(): T = ...
| |next // is expanded to next()
|
| In Dotty, this idiom is an error. The application syntax has to follow exactly the parameter syntax.
| Excluded from this rule are methods that are defined in Java or that override methods defined in Java.
-----------------------------------------------------------------------------
1 error found
```
### 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 #43472 from LuciferYang/SPARK-45610.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
index 3800dca..b22bd74 100644
--- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
+++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
@@ -168,7 +168,7 @@
test("reading from multiple paths") {
val df = spark.read.format("avro").load(episodesAvro, episodesAvro)
- assert(df.count == 16)
+ assert(df.count() == 16)
}
test("reading and writing partitioned data") {
@@ -197,7 +197,7 @@
withTempPath { dir =>
val df = spark.read.format("avro").load(episodesAvro)
df.write.parquet(dir.getCanonicalPath)
- assert(spark.read.parquet(dir.getCanonicalPath).count() === df.count)
+ assert(spark.read.parquet(dir.getCanonicalPath).count() === df.count())
}
}
@@ -549,7 +549,7 @@
Row(null, null, null, null, null)))
val df = spark.createDataFrame(rdd, schema)
df.write.format("avro").save(dir.toString)
- assert(spark.read.format("avro").load(dir.toString).count == rdd.count)
+ assert(spark.read.format("avro").load(dir.toString).count() == rdd.count())
}
}
@@ -568,7 +568,7 @@
))
val df = spark.createDataFrame(rdd, schema)
df.write.format("avro").save(dir.toString)
- assert(spark.read.format("avro").load(dir.toString).count == rdd.count)
+ assert(spark.read.format("avro").load(dir.toString).count() == rdd.count())
}
}
@@ -628,7 +628,7 @@
))
val df = spark.createDataFrame(rdd, schema)
df.write.format("avro").save(dir.toString)
- assert(spark.read.format("avro").load(dir.toString).count == rdd.count)
+ assert(spark.read.format("avro").load(dir.toString).count() == rdd.count())
checkAnswer(
spark.read.format("avro").load(dir.toString).select("date"),
Seq(Row(null), Row(new Date(1451865600000L)), Row(new Date(1459987200000L))))
@@ -666,7 +666,7 @@
Array[Row](Row("Bobby G. can't swim")))))
val df = spark.createDataFrame(rdd, testSchema)
df.write.format("avro").save(dir.toString)
- assert(spark.read.format("avro").load(dir.toString).count == rdd.count)
+ assert(spark.read.format("avro").load(dir.toString).count() == rdd.count())
}
}
@@ -1021,7 +1021,7 @@
val currentDate = new Date(System.currentTimeMillis())
val schema = StructType(Seq(
StructField("_1", DateType, false), StructField("_2", TimestampType, false)))
- val writeDs = Seq((currentDate, currentTime)).toDS
+ val writeDs = Seq((currentDate, currentTime)).toDS()
val avroDir = tempDir + "/avro"
writeDs.write.format("avro").save(avroDir)
@@ -1050,12 +1050,12 @@
StructField("_1", DateType, nullable = true),
StructField("_2", TimestampType, nullable = true))
)
- val writeDs = Seq((nullDate, nullTime)).toDS
+ val writeDs = Seq((nullDate, nullTime)).toDS()
val avroDir = tempDir + "/avro"
writeDs.write.format("avro").save(avroDir)
val readValues =
- spark.read.schema(schema).format("avro").load(avroDir).as[(Date, Timestamp)].collect
+ spark.read.schema(schema).format("avro").load(avroDir).as[(Date, Timestamp)].collect()
assert(readValues.size == 1)
assert(readValues.head == ((nullDate, nullTime)))
@@ -1147,7 +1147,7 @@
val result = spark
.read
.option("avroSchema", avroSchema)
- .format("avro").load(testAvro).select("missingField").first
+ .format("avro").load(testAvro).select("missingField").first()
assert(result === Row("foo"))
}
@@ -1743,13 +1743,13 @@
// Test if load works as expected
withTempPath { tempDir =>
val df = spark.read.format("avro").load(episodesAvro)
- assert(df.count == 8)
+ assert(df.count() == 8)
val tempSaveDir = s"$tempDir/save/"
df.write.format("avro").save(tempSaveDir)
val newDf = spark.read.format("avro").load(tempSaveDir)
- assert(newDf.count == 8)
+ assert(newDf.count() == 8)
}
}
@@ -1757,7 +1757,7 @@
// Test if load works as expected
withTempPath { tempDir =>
val df = spark.read.format("avro").load(episodesAvro)
- assert(df.count == 8)
+ assert(df.count() == 8)
val tempSaveDir = s"$tempDir/save/"
df.write.format("avro").save(tempSaveDir)
@@ -1928,11 +1928,11 @@
test("read avro file partitioned") {
withTempPath { dir =>
- val df = (0 to 1024 * 3).toDS.map(i => s"record${i}").toDF("records")
+ val df = (0 to 1024 * 3).toDS().map(i => s"record${i}").toDF("records")
val outputDir = s"$dir/${UUID.randomUUID}"
df.write.format("avro").save(outputDir)
val input = spark.read.format("avro").load(outputDir)
- assert(input.collect.toSet.size === 1024 * 3 + 1)
+ assert(input.collect().toSet.size === 1024 * 3 + 1)
assert(input.rdd.partitions.size > 2)
}
}
@@ -2057,21 +2057,21 @@
val fileWithoutExtension = s"${dir.getCanonicalPath}/episodes"
val df1 = spark.read.format("avro").load(fileWithoutExtension)
- assert(df1.count == 8)
+ assert(df1.count() == 8)
val schema = new StructType()
.add("title", StringType)
.add("air_date", StringType)
.add("doctor", IntegerType)
val df2 = spark.read.schema(schema).format("avro").load(fileWithoutExtension)
- assert(df2.count == 8)
+ assert(df2.count() == 8)
}
}
test("SPARK-24836: checking the ignoreExtension option") {
withTempPath { tempDir =>
val df = spark.read.format("avro").load(episodesAvro)
- assert(df.count == 8)
+ assert(df.count() == 8)
val tempSaveDir = s"$tempDir/save/"
df.write.format("avro").save(tempSaveDir)
@@ -2084,7 +2084,7 @@
.format("avro")
.load(tempSaveDir)
- assert(newDf.count == 8)
+ assert(newDf.count() == 8)
}
}
diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala
index aa0d713..80f0d6b 100644
--- a/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala
+++ b/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala
@@ -64,7 +64,7 @@
withTempPath { dir =>
withTempTable("t1", "avroTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
@@ -83,7 +83,7 @@
withTempPath { dir =>
withTempTable("t1", "avroTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(
dir,
@@ -104,7 +104,7 @@
withTempPath { dir =>
withTempTable("t1", "avroTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
@@ -177,7 +177,7 @@
import spark.implicits._
val middle = width / 2
val selectExpr = (1 to width).map(i => s"value as c$i")
- spark.range(values).map(_ => Random.nextLong).toDF()
+ spark.range(values).map(_ => Random.nextLong()).toDF()
.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT * FROM t1"))
@@ -200,7 +200,7 @@
import spark.implicits._
val middle = width / 2
val selectExpr = (1 to width).map(i => s"value as c$i")
- spark.range(values).map(_ => Random.nextLong).toDF()
+ spark.range(values).map(_ => Random.nextLong()).toDF()
.selectExpr(selectExpr: _*)
.repartition(files) // ensure at least `files` number of splits (but maybe more)
.createOrReplaceTempView("t1")
diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala b/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala
index d1db290..e61ac43 100644
--- a/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala
+++ b/connector/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala
@@ -47,7 +47,7 @@
val selectExpr = (1 to width).map(i => s"value as c$i")
// repartition to ensure we will write multiple files
val df = spark.range(values)
- .map(_ => Random.nextInt).selectExpr(selectExpr: _*).repartition(files)
+ .map(_ => Random.nextInt()).selectExpr(selectExpr: _*).repartition(files)
.persist(StorageLevel.DISK_ONLY)
// cache the data to ensure we are not benchmarking range or repartition
df.noop()
@@ -55,7 +55,7 @@
val benchmark = new Benchmark(s"Write wide rows into $files files", values, output = output)
benchmark.addCase("Write wide rows") { _ =>
spark.sql("SELECT * FROM t1").
- write.format("avro").save(s"${dir.getCanonicalPath}/${Random.nextLong.abs}")
+ write.format("avro").save(s"${dir.getCanonicalPath}/${Random.nextLong().abs}")
}
benchmark.run()
}
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
index 04d284f..d9a77f2 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
@@ -63,7 +63,7 @@
})
val ex = intercept[SparkException] {
- Seq("1").toDS.withColumn("udf_val", throwException($"value")).collect()
+ Seq("1").toDS().withColumn("udf_val", throwException($"value")).collect()
}
assert(ex.getCause.isInstanceOf[SparkException])
@@ -103,7 +103,7 @@
udf((_: String) => throw new SparkException("test" * 10000))
val ex = intercept[SparkException] {
- Seq("1").toDS.withColumn("udf_val", throwException($"value")).collect()
+ Seq("1").toDS().withColumn("udf_val", throwException($"value")).collect()
}
assert(ex.getErrorClass != null)
@@ -1011,9 +1011,9 @@
assert(df1.sameSemantics(df3) === false)
assert(df3.sameSemantics(df4) === true)
- assert(df1.semanticHash === df2.semanticHash)
- assert(df1.semanticHash !== df3.semanticHash)
- assert(df3.semanticHash === df4.semanticHash)
+ assert(df1.semanticHash() === df2.semanticHash())
+ assert(df1.semanticHash() !== df3.semanticHash())
+ assert(df3.semanticHash() === df4.semanticHash())
}
test("toJSON") {
@@ -1414,7 +1414,7 @@
val r4 = uuid()
val r5 = shuffle(col("a"))
df.select(r, r.as("r"), r2, r2.as("r2"), r3, r3.as("r3"), r4, r4.as("r4"), r5, r5.as("r5"))
- .collect
+ .collect()
.foreach { row =>
(0 until 5).foreach(i => assert(row.get(i * 2) === row.get(i * 2 + 1)))
}
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionSuite.scala
index ef1cf78..b77e929 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionSuite.scala
@@ -293,7 +293,7 @@
val df = createDFWithNestedColumns
// Rows with the specified nested columns whose null values are dropped.
- assert(df.count == 3)
+ assert(df.count() == 3)
checkAnswer(df.na.drop("any", Seq("c1.c1-1")), Seq(Row(Row("b1", "b2"))))
}
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala
index 609fad5..bd0aabc 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala
@@ -340,7 +340,7 @@
assert(kvgds == null)
i + 1
}
- val result = df.select(f($"id")).as[Long].head
+ val result = df.select(f($"id")).as[Long].head()
assert(result == 1L)
}
}
diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala
index 2f8eacb..7401164 100644
--- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala
+++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala
@@ -171,7 +171,7 @@
return
}
- val promise = Promise[Seq[ArtifactSummary]]
+ val promise = Promise[Seq[ArtifactSummary]]()
val responseHandler = new StreamObserver[proto.AddArtifactsResponse] {
private val summaries = mutable.Buffer.empty[ArtifactSummary]
override def onNext(v: AddArtifactsResponse): Unit = {
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 74c8423..0f8178c 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
@@ -75,7 +75,7 @@
}
override def next(): U = {
- retryIter(_.next)
+ retryIter(_.next())
}
override def hasNext: Boolean = {
diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala
index e99e3a9..3c416bb 100644
--- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala
+++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala
@@ -183,10 +183,10 @@
throw new SparkSQLException(
errorClass = "INVALID_CURSOR.POSITION_NOT_AVAILABLE",
messageParameters = Map("index" -> index.toString, "responseId" -> responseId))
- } else if (getLastResponseIndex.exists(index > _)) {
+ } else if (getLastResponseIndex().exists(index > _)) {
// If index > lastIndex, it's out of bounds. This is an internal error.
throw new IllegalStateException(
- s"Cursor position $index is beyond last index $getLastResponseIndex.")
+ s"Cursor position $index is beyond last index ${getLastResponseIndex()}.")
}
ret
}
diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala
index ddad7da..a3ce813 100644
--- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala
+++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala
@@ -57,7 +57,7 @@
s"Illegal operation type ${request.getPlan.getOpTypeCase} to be handled here.")
}
val planner = new SparkConnectPlanner(executeHolder)
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker
+ val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val dataframe =
Dataset.ofRows(
sessionHolder.session,
diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
index 299f4f8..f5d83b8 100644
--- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
+++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
@@ -2514,7 +2514,7 @@
val namedArguments = getSqlCommand.getNamedArgumentsMap
val posArgs = getSqlCommand.getPosArgsList
val posArguments = getSqlCommand.getPosArgumentsList
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker
+ val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val df = if (!namedArguments.isEmpty) {
session.sql(
getSqlCommand.getSql,
@@ -2724,7 +2724,7 @@
replace = createView.getReplace,
viewType = viewType)
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker
+ val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
Dataset.ofRows(session, plan, tracker).queryExecution.commandExecuted
executeHolder.eventsManager.postFinished()
}
@@ -2742,7 +2742,7 @@
// Transform the input plan into the logical plan.
val plan = transformRelation(writeOperation.getInput)
// And create a Dataset from the plan.
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker
+ val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val dataset = Dataset.ofRows(session, plan, tracker)
val w = dataset.write
@@ -2814,7 +2814,7 @@
// Transform the input plan into the logical plan.
val plan = transformRelation(writeOperation.getInput)
// And create a Dataset from the plan.
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker
+ val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val dataset = Dataset.ofRows(session, plan, tracker)
val w = dataset.writeTo(table = writeOperation.getTableName)
@@ -2873,7 +2873,7 @@
writeOp: WriteStreamOperationStart,
responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
val plan = transformRelation(writeOp.getInput)
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker
+ val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val dataset = Dataset.ofRows(session, plan, tracker)
// Call manually as writeStream does not trigger ReadyForExecution
tracker.setReadyForExecution()
@@ -3082,7 +3082,7 @@
val exception_builder = StreamingQueryCommandResult.ExceptionResult
.newBuilder()
exception_builder
- .setExceptionMessage(e.toString)
+ .setExceptionMessage(e.toString())
.setErrorClass(e.getErrorClass)
val stackTrace = Option(ExceptionUtils.getStackTrace(e))
diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala
index 9e8a945..2430716 100644
--- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala
+++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala
@@ -245,7 +245,7 @@
postAnalyzed(Some(analyzedPlan))
}
- def readyForExecution(tracker: QueryPlanningTracker): Unit = postReadyForExecution
+ def readyForExecution(tracker: QueryPlanningTracker): Unit = postReadyForExecution()
}))
}
diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala
index fbaa9e0..3b42b58 100644
--- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala
+++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala
@@ -31,7 +31,7 @@
def main(args: Array[String]): Unit = {
// Set the active Spark Session, and starts SparkEnv instance (via Spark Context)
logInfo("Starting Spark session.")
- val session = SparkSession.builder.getOrCreate()
+ val session = SparkSession.builder().getOrCreate()
try {
try {
SparkConnectService.start(session.sparkContext)
diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala
index abbb181..ce45262 100644
--- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala
+++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala
@@ -606,7 +606,7 @@
}
override def onError(throwable: Throwable): Unit = {
- verifyEvents.onCanceled
+ verifyEvents.onCanceled()
}
override def onCompleted(): Unit = {
diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala
index 7dedf5e..2a65032 100644
--- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala
+++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala
@@ -184,7 +184,7 @@
}
test("single chunk artifact") {
- val promise = Promise[AddArtifactsResponse]
+ val promise = Promise[AddArtifactsResponse]()
val handler = new TestAddArtifactsHandler(new DummyStreamObserver(promise))
try {
val name = "classes/smallClassFile.class"
@@ -208,7 +208,7 @@
}
test("Multi chunk artifact") {
- val promise = Promise[AddArtifactsResponse]
+ val promise = Promise[AddArtifactsResponse]()
val handler = new TestAddArtifactsHandler(new DummyStreamObserver(promise))
try {
val name = "jars/junitLargeJar.jar"
@@ -232,7 +232,7 @@
}
test("Mix of single-chunk and chunked artifacts") {
- val promise = Promise[AddArtifactsResponse]
+ val promise = Promise[AddArtifactsResponse]()
val handler = new TestAddArtifactsHandler(new DummyStreamObserver(promise))
try {
val names = Seq(
@@ -272,7 +272,7 @@
}
test("Artifacts that fail CRC are not added to the artifact manager") {
- val promise = Promise[AddArtifactsResponse]
+ val promise = Promise[AddArtifactsResponse]()
val handler = new TestAddArtifactsHandler(new DummyStreamObserver(promise))
try {
val name = "classes/smallClassFile.class"
@@ -365,7 +365,7 @@
}
test("Artifacts names are not allowed to be absolute paths") {
- val promise = Promise[AddArtifactsResponse]
+ val promise = Promise[AddArtifactsResponse]()
val handler = new TestAddArtifactsHandler(new DummyStreamObserver(promise))
try {
val name = "/absolute/path/"
@@ -382,7 +382,7 @@
}
test("Artifact name/paths cannot reference parent/sibling/nephew directories") {
- val promise = Promise[AddArtifactsResponse]
+ val promise = Promise[AddArtifactsResponse]()
val handler = new TestAddArtifactsHandler(new DummyStreamObserver(promise))
try {
val names = Seq("..", "../sibling", "../nephew/directory", "a/../../b", "x/../y/../..")
diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala
index b7a92fa..54f7396 100644
--- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala
+++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala
@@ -38,7 +38,7 @@
class ArtifactStatusesHandlerSuite extends SharedSparkSession with ResourceHelper {
def getStatuses(names: Seq[String], exist: Set[String]): ArtifactStatusesResponse = {
- val promise = Promise[ArtifactStatusesResponse]
+ val promise = Promise[ArtifactStatusesResponse]()
val handler = new SparkConnectArtifactStatusesHandler(new DummyStreamObserver(promise)) {
override protected def cacheExists(
userId: String,
diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/FetchErrorDetailsHandlerSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/FetchErrorDetailsHandlerSuite.scala
index 1b908ca..40439a2 100644
--- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/FetchErrorDetailsHandlerSuite.scala
+++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/FetchErrorDetailsHandlerSuite.scala
@@ -51,7 +51,7 @@
userId: String,
sessionId: String,
errorId: String): FetchErrorDetailsResponse = {
- val promise = Promise[FetchErrorDetailsResponse]
+ val promise = Promise[FetchErrorDetailsResponse]()
val handler =
new SparkConnectFetchErrorDetailsHandler(new FetchErrorDetailsResponseObserver(promise))
val context = proto.UserContext
diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala
index 33f6627..8f76d58 100644
--- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala
+++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala
@@ -130,7 +130,7 @@
"org.apache.spark.sql.connect.service.TestingInterceptorNoTrivialCtor") {
checkError(
exception = intercept[SparkException] {
- SparkConnectInterceptorRegistry.createConfiguredInterceptors
+ SparkConnectInterceptorRegistry.createConfiguredInterceptors()
},
errorClass = "CONNECT.INTERCEPTOR_CTOR_MISSING",
parameters =
@@ -142,7 +142,7 @@
"org.apache.spark.sql.connect.service.TestingInterceptorInstantiationError") {
checkError(
exception = intercept[SparkException] {
- SparkConnectInterceptorRegistry.createConfiguredInterceptors
+ SparkConnectInterceptorRegistry.createConfiguredInterceptors()
},
errorClass = "CONNECT.INTERCEPTOR_RUNTIME_ERROR",
parameters = Map("msg" -> "Bad Error"))
@@ -151,10 +151,10 @@
test("No configured interceptors returns empty list") {
// Not set.
- assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors.isEmpty)
+ assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors().isEmpty)
// Set to empty string
withSparkConf(Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> "") {
- assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors.isEmpty)
+ assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors().isEmpty)
}
}
@@ -163,14 +163,14 @@
Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key ->
(" org.apache.spark.sql.connect.service.DummyInterceptor," +
" org.apache.spark.sql.connect.service.DummyInterceptor ")) {
- assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors.size == 2)
+ assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors().size == 2)
}
}
test("Configured class not found is properly thrown") {
withSparkConf(Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> "this.class.does.not.exist") {
assertThrows[ClassNotFoundException] {
- SparkConnectInterceptorRegistry.createConfiguredInterceptors
+ SparkConnectInterceptorRegistry.createConfiguredInterceptors()
}
}
}
diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala
index e425151..e4ee7ab 100644
--- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala
+++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala
@@ -187,7 +187,7 @@
.option("url", jdbcUrl)
.option("query", query)
.load()
- assert(df.collect.toSet === expectedResult)
+ assert(df.collect().toSet === expectedResult)
// query option in the create table path.
sql(
@@ -196,7 +196,7 @@
|USING org.apache.spark.sql.jdbc
|OPTIONS (url '$jdbcUrl', query '$query')
""".stripMargin.replaceAll("\n", " "))
- assert(sql("select x, y from queryOption").collect.toSet == expectedResult)
+ assert(sql("select x, y from queryOption").collect().toSet == expectedResult)
}
test("SPARK-30062") {
@@ -210,10 +210,10 @@
for (_ <- 0 to 2) {
df.write.mode(SaveMode.Append).jdbc(jdbcUrl, "tblcopy", new Properties)
}
- assert(sqlContext.read.jdbc(jdbcUrl, "tblcopy", new Properties).count === 6)
+ assert(sqlContext.read.jdbc(jdbcUrl, "tblcopy", new Properties).count() === 6)
df.write.mode(SaveMode.Overwrite).option("truncate", true)
.jdbc(jdbcUrl, "tblcopy", new Properties)
- val actual = sqlContext.read.jdbc(jdbcUrl, "tblcopy", new Properties).collect
+ val actual = sqlContext.read.jdbc(jdbcUrl, "tblcopy", new Properties).collect()
assert(actual.length === 2)
assert(actual.toSet === expectedResult)
}
diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala
index 455416e..80ef33b 100644
--- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala
+++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala
@@ -399,7 +399,7 @@
.option("prepareQuery", prepareQuery)
.option("query", query)
.load()
- assert(df.collect.toSet === expectedResult)
+ assert(df.collect().toSet === expectedResult)
}
test("SPARK-37259: prepareQuery and dbtable JDBC options") {
@@ -417,7 +417,7 @@
.option("prepareQuery", prepareQuery)
.option("dbtable", dbtable)
.load()
- assert(df.collect.toSet === expectedResult)
+ assert(df.collect().toSet === expectedResult)
}
test("SPARK-37259: temp table prepareQuery and query JDBC options") {
@@ -435,6 +435,6 @@
.option("prepareQuery", prepareQuery)
.option("query", query)
.load()
- assert(df.collect.toSet === expectedResult)
+ assert(df.collect().toSet === expectedResult)
}
}
diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
index dc3acb6..4294041 100644
--- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
+++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
@@ -183,7 +183,7 @@
.option("url", jdbcUrl)
.option("query", query)
.load()
- assert(df.collect.toSet === expectedResult)
+ assert(df.collect().toSet === expectedResult)
// query option in the create table path.
sql(
@@ -192,6 +192,6 @@
|USING org.apache.spark.sql.jdbc
|OPTIONS (url '$jdbcUrl', query '$query')
""".stripMargin.replaceAll("\n", " "))
- assert(sql("select x, y from queryOption").collect.toSet == expectedResult)
+ assert(sql("select x, y from queryOption").collect().toSet == expectedResult)
}
}
diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
index 4378c69..102a85e 100644
--- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
+++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
@@ -460,7 +460,7 @@
""""D" >= '2018-07-11' AND "D" < '2018-07-15'""",
""""D" >= '2018-07-15'"""))
}
- assert(df1.collect.toSet === expectedResult)
+ assert(df1.collect().toSet === expectedResult)
// TimestampType partition column
val df2 = spark.read.format("jdbc")
@@ -482,7 +482,7 @@
""""T" < '2018-07-15 20:50:32.5' or "T" is null""",
""""T" >= '2018-07-15 20:50:32.5'"""))
}
- assert(df2.collect.toSet === expectedResult)
+ assert(df2.collect().toSet === expectedResult)
}
test("query JDBC option") {
@@ -499,7 +499,7 @@
.option("query", query)
.option("oracle.jdbc.mapDateToTimestamp", "false")
.load()
- assert(df.collect.toSet === expectedResult)
+ assert(df.collect().toSet === expectedResult)
// query option in the create table path.
sql(
@@ -510,7 +510,7 @@
| query '$query',
| oracle.jdbc.mapDateToTimestamp false)
""".stripMargin.replaceAll("\n", " "))
- assert(sql("select id, d, t from queryOption").collect.toSet == expectedResult)
+ assert(sql("select id, d, t from queryOption").collect().toSet == expectedResult)
}
test("SPARK-32992: map Oracle's ROWID type to StringType") {
diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
index 90d6f6a..e4b0535 100644
--- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
+++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
@@ -344,7 +344,7 @@
.option("url", jdbcUrl)
.option("query", query)
.load()
- assert(df.collect.toSet === expectedResult)
+ assert(df.collect().toSet === expectedResult)
// query option in the create table path.
sql(
@@ -353,7 +353,7 @@
|USING org.apache.spark.sql.jdbc
|OPTIONS (url '$jdbcUrl', query '$query')
""".stripMargin.replaceAll("\n", " "))
- assert(sql("select c1, c3 from queryOption").collect.toSet == expectedResult)
+ assert(sql("select c1, c3 from queryOption").collect().toSet == expectedResult)
}
test("write byte as smallint") {
diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala
index 6198b64..e83a2c9 100644
--- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala
+++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala
@@ -535,7 +535,7 @@
}
KafkaOffsetRange(tp, fromOffset, untilOffset, preferredLoc = None)
}
- rangeCalculator.getRanges(ranges, getSortedExecutorList)
+ rangeCalculator.getRanges(ranges, getSortedExecutorList())
}
private def partitionsAssignedToConsumer(
diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala
index 245700d..1b1e0f1 100644
--- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala
+++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala
@@ -701,7 +701,7 @@
def acquire(
topicPartition: TopicPartition,
kafkaParams: ju.Map[String, Object]): KafkaDataConsumer = {
- if (TaskContext.get != null && TaskContext.get.attemptNumber >= 1) {
+ if (TaskContext.get() != null && TaskContext.get().attemptNumber() >= 1) {
val cacheKey = new CacheKey(topicPartition, kafkaParams)
// If this is reattempt at running the task, then invalidate cached consumer if any.
diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
index 6801d14..e42662c 100644
--- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
+++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
@@ -65,7 +65,7 @@
eventually(timeout(streamingTimeout)) {
// Should read all committed messages
- checkAnswer(spark.table(table), (1 to 5).toDF)
+ checkAnswer(spark.table(table), (1 to 5).toDF())
}
producer.beginTransaction()
@@ -75,7 +75,7 @@
producer.abortTransaction()
// Should not read aborted messages
- checkAnswer(spark.table(table), (1 to 5).toDF)
+ checkAnswer(spark.table(table), (1 to 5).toDF())
producer.beginTransaction()
(11 to 15).foreach { i =>
@@ -85,7 +85,7 @@
eventually(timeout(streamingTimeout)) {
// Should skip aborted messages and read new committed ones.
- checkAnswer(spark.table(table), ((1 to 5) ++ (11 to 15)).toDF)
+ checkAnswer(spark.table(table), ((1 to 5) ++ (11 to 15)).toDF())
}
} finally {
q.stop()
@@ -126,14 +126,14 @@
eventually(timeout(streamingTimeout)) {
// Should read uncommitted messages
- checkAnswer(spark.table(table), (1 to 5).toDF)
+ checkAnswer(spark.table(table), (1 to 5).toDF())
}
producer.commitTransaction()
eventually(timeout(streamingTimeout)) {
// Should read all committed messages
- checkAnswer(spark.table(table), (1 to 5).toDF)
+ checkAnswer(spark.table(table), (1 to 5).toDF())
}
producer.beginTransaction()
@@ -144,7 +144,7 @@
eventually(timeout(streamingTimeout)) {
// Should read aborted messages
- checkAnswer(spark.table(table), (1 to 10).toDF)
+ checkAnswer(spark.table(table), (1 to 10).toDF())
}
producer.beginTransaction()
@@ -154,14 +154,14 @@
eventually(timeout(streamingTimeout)) {
// Should read all messages including committed, aborted and uncommitted messages
- checkAnswer(spark.table(table), (1 to 15).toDF)
+ checkAnswer(spark.table(table), (1 to 15).toDF())
}
producer.commitTransaction()
eventually(timeout(streamingTimeout)) {
// Should read all messages including committed and aborted messages
- checkAnswer(spark.table(table), (1 to 15).toDF)
+ checkAnswer(spark.table(table), (1 to 15).toDF())
}
} finally {
q.stop()
diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index 2315147..4e8da13 100644
--- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -248,7 +248,7 @@
.option("subscribe", outputTopic)
.load()
.select(expr("CAST(value AS string)"))
- .toDF
+ .toDF()
.collect().map(_.getAs[String]("value")).toList
}
@@ -439,7 +439,7 @@
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
.option("subscribe", topic)
- testStream(reader.load)(
+ testStream(reader.load())(
makeSureGetOffsetCalled,
StopStream,
StartStream(),
@@ -853,7 +853,7 @@
true
},
AssertOnQuery { q =>
- val latestOffset: Option[(Long, OffsetSeq)] = q.offsetLog.getLatest
+ val latestOffset: Option[(Long, OffsetSeq)] = q.offsetLog.getLatest()
latestOffset.exists { offset =>
!offset._2.offsets.exists(_.exists(_.json == "{}"))
}
@@ -2499,7 +2499,7 @@
.trigger(defaultTrigger)
.start()
eventually(timeout(streamingTimeout)) {
- assert(spark.table("kafkaColumnTypes").count == 1,
+ assert(spark.table("kafkaColumnTypes").count() == 1,
s"Unexpected results: ${spark.table("kafkaColumnTypes").collectAsList()}")
}
val row = spark.table("kafkaColumnTypes").head()
diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
index 6e1ece3..8d4e3e5 100644
--- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
+++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
@@ -97,11 +97,11 @@
// Specify explicit earliest and latest offset values
val df = createDF(topic,
withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest"))
- checkAnswer(df, (0 to 20).map(_.toString).toDF)
+ checkAnswer(df, (0 to 20).map(_.toString).toDF())
// "latest" should late bind to the current (latest) offset in the df
testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2))
- checkAnswer(df, (0 to 29).map(_.toString).toDF)
+ checkAnswer(df, (0 to 29).map(_.toString).toDF())
}
test("default starting and ending offsets") {
@@ -114,7 +114,7 @@
// Implicit offset values, should default to earliest and latest
val df = createDF(topic)
// Test that we default to "earliest" and "latest"
- checkAnswer(df, (0 to 20).map(_.toString).toDF)
+ checkAnswer(df, (0 to 20).map(_.toString).toDF())
}
test("explicit offsets") {
@@ -140,15 +140,15 @@
val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets)
val df = createDF(topic,
withOptions = Map("startingOffsets" -> startingOffsets, "endingOffsets" -> endingOffsets))
- checkAnswer(df, (0 to 20).map(_.toString).toDF)
+ checkAnswer(df, (0 to 20).map(_.toString).toDF())
// static offset partition 2, nothing should change
testUtils.sendMessages(topic, (31 to 39).map(_.toString).toArray, Some(2))
- checkAnswer(df, (0 to 20).map(_.toString).toDF)
+ checkAnswer(df, (0 to 20).map(_.toString).toDF())
// latest offset partition 1, should change
testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(1))
- checkAnswer(df, (0 to 30).map(_.toString).toDF)
+ checkAnswer(df, (0 to 30).map(_.toString).toDF())
}
test("default starting and ending offsets with headers") {
@@ -171,7 +171,7 @@
// Test that we default to "earliest" and "latest"
checkAnswer(df, Seq(("1", null),
("2", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))),
- ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8))))).toDF)
+ ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8))))).toDF())
}
test("timestamp provided for starting and ending") {
@@ -393,7 +393,7 @@
.option("subscribe", topic)
val df2 = optionFn(df).load().selectExpr("CAST(value AS STRING)")
- checkAnswer(df2, expectation.map(_.toString).toDF)
+ checkAnswer(df2, expectation.map(_.toString).toDF())
}
test("reuse same dataframe in query") {
@@ -405,7 +405,7 @@
// Specify explicit earliest and latest offset values
val df = createDF(topic,
withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest"))
- checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF)
+ checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF())
}
test("test late binding start offsets") {
@@ -432,13 +432,13 @@
val df = createDF(topic,
withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest"),
Some(kafkaUtils.brokerAddress))
- checkAnswer(df, (0 to 9).map(_.toString).toDF)
+ checkAnswer(df, (0 to 9).map(_.toString).toDF())
// Blow away current set of messages.
kafkaUtils.cleanupLogs()
// Add some more data, but do not call cleanup
kafkaUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(0))
// Ensure that we late bind to the new starting position
- checkAnswer(df, (10 to 19).map(_.toString).toDF)
+ checkAnswer(df, (10 to 19).map(_.toString).toDF())
} finally {
if (kafkaUtils != null) {
kafkaUtils.teardown()
@@ -521,7 +521,7 @@
// Should read all committed messages
testUtils.waitUntilOffsetAppears(new TopicPartition(topic, 0), 6)
- checkAnswer(df, (1 to 5).map(_.toString).toDF)
+ checkAnswer(df, (1 to 5).map(_.toString).toDF())
producer.beginTransaction()
(6 to 10).foreach { i =>
@@ -531,7 +531,7 @@
// Should not read aborted messages
testUtils.waitUntilOffsetAppears(new TopicPartition(topic, 0), 12)
- checkAnswer(df, (1 to 5).map(_.toString).toDF)
+ checkAnswer(df, (1 to 5).map(_.toString).toDF())
producer.beginTransaction()
(11 to 15).foreach { i =>
@@ -541,7 +541,7 @@
// Should skip aborted messages and read new committed ones.
testUtils.waitUntilOffsetAppears(new TopicPartition(topic, 0), 18)
- checkAnswer(df, ((1 to 5) ++ (11 to 15)).map(_.toString).toDF)
+ checkAnswer(df, ((1 to 5) ++ (11 to 15)).map(_.toString).toDF())
}
}
@@ -565,13 +565,13 @@
// "read_uncommitted" should see all messages including uncommitted ones
testUtils.waitUntilOffsetAppears(new TopicPartition(topic, 0), 5)
- checkAnswer(df, (1 to 5).map(_.toString).toDF)
+ checkAnswer(df, (1 to 5).map(_.toString).toDF())
producer.commitTransaction()
// Should read all committed messages
testUtils.waitUntilOffsetAppears(new TopicPartition(topic, 0), 6)
- checkAnswer(df, (1 to 5).map(_.toString).toDF)
+ checkAnswer(df, (1 to 5).map(_.toString).toDF())
producer.beginTransaction()
(6 to 10).foreach { i =>
@@ -581,7 +581,7 @@
// "read_uncommitted" should see all messages including uncommitted or aborted ones
testUtils.waitUntilOffsetAppears(new TopicPartition(topic, 0), 12)
- checkAnswer(df, (1 to 10).map(_.toString).toDF)
+ checkAnswer(df, (1 to 10).map(_.toString).toDF())
producer.beginTransaction()
(11 to 15).foreach { i =>
@@ -591,7 +591,7 @@
// Should read all messages
testUtils.waitUntilOffsetAppears(new TopicPartition(topic, 0), 18)
- checkAnswer(df, (1 to 15).map(_.toString).toDF)
+ checkAnswer(df, (1 to 15).map(_.toString).toDF())
}
}
diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumerSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumerSuite.scala
index 8c5289a..d494433 100644
--- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumerSuite.scala
+++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumerSuite.scala
@@ -197,7 +197,7 @@
@volatile var error: Throwable = null
def consume(i: Int): Unit = {
- val taskContext = if (Random.nextBoolean) {
+ val taskContext = if (Random.nextBoolean()) {
new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), 1,
null, null, null)
} else {
diff --git a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala
index 93cc522..497ba03 100644
--- a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala
+++ b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala
@@ -195,7 +195,7 @@
kerberosServiceName: String): String = {
val params =
s"""
- |${SecurityUtils.getKrb5LoginModuleName} required
+ |${SecurityUtils.getKrb5LoginModuleName()} required
| debug=${SecurityUtils.isGlobalKrbDebugEnabled()}
| useKeyTab=true
| serviceName="$kerberosServiceName"
@@ -209,7 +209,7 @@
private def getTicketCacheJaasParams(clusterConf: KafkaTokenClusterConf): String = {
val params =
s"""
- |${SecurityUtils.getKrb5LoginModuleName} required
+ |${SecurityUtils.getKrb5LoginModuleName()} required
| debug=${SecurityUtils.isGlobalKrbDebugEnabled()}
| useTicketCache=true
| serviceName="${clusterConf.kerberosServiceName}";
diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala
index 971b3b3..d803726 100644
--- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala
+++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala
@@ -83,7 +83,7 @@
}
protected def getBrokers = {
- val c = consumer
+ val c = consumer()
val result = new ju.HashMap[TopicPartition, String]()
val hosts = new ju.HashMap[TopicPartition, String]()
val assignments = c.assignment().iterator()
@@ -187,7 +187,7 @@
* Returns the latest (highest) available offsets, taking new partitions into account.
*/
protected def latestOffsets(): Map[TopicPartition, Long] = {
- val c = consumer
+ val c = consumer()
paranoidPoll(c)
val parts = c.assignment().asScala
@@ -247,7 +247,7 @@
val metadata = Map(
"offsets" -> offsetRanges.toList,
StreamInputInfo.METADATA_KEY_DESCRIPTION -> description)
- val inputInfo = StreamInputInfo(id, rdd.count, metadata)
+ val inputInfo = StreamInputInfo(id, rdd.count(), metadata)
ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo)
currentOffsets = untilOffsets
@@ -256,7 +256,7 @@
}
override def start(): Unit = {
- val c = consumer
+ val c = consumer()
paranoidPoll(c)
if (currentOffsets.isEmpty) {
currentOffsets = c.assignment().asScala.map { tp =>
@@ -296,14 +296,14 @@
val m = new ju.HashMap[TopicPartition, OffsetAndMetadata]()
var osr = commitQueue.poll()
while (null != osr) {
- val tp = osr.topicPartition
+ val tp = osr.topicPartition()
val x = m.get(tp)
val offset = if (null == x) { osr.untilOffset } else { Math.max(x.offset, osr.untilOffset) }
m.put(tp, new OffsetAndMetadata(offset))
osr = commitQueue.poll()
}
if (!m.isEmpty) {
- consumer.commitAsync(m, commitCallback.get)
+ consumer().commitAsync(m, commitCallback.get)
}
}
diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala
index 8becbe4..c7ac6a8 100644
--- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala
+++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala
@@ -293,7 +293,7 @@
lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](topicPartition, kafkaParams)
- if (context != null && context.attemptNumber >= 1) {
+ if (context != null && context.attemptNumber() >= 1) {
// If this is reattempt at running the task, then invalidate cached consumers if any and
// start with a new one. If prior attempt failures were cache related then this way old
// problematic consumers can be removed.
diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
index b6ce5e2..b8dbfe2 100644
--- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
+++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
@@ -87,7 +87,7 @@
if (compacted) {
super.count()
} else {
- offsetRanges.map(_.count).sum
+ offsetRanges.map(_.count()).sum
}
override def countApprox(
@@ -97,7 +97,7 @@
if (compacted) {
super.countApprox(timeout, confidence)
} else {
- val c = count
+ val c = count()
new PartialResult(new BoundedDouble(c, 1.0, c, c), true)
}
@@ -105,7 +105,7 @@
if (compacted) {
super.isEmpty()
} else {
- count == 0L
+ count() == 0L
}
override def take(num: Int): Array[ConsumerRecord[K, V]] =
@@ -116,7 +116,7 @@
} else {
val nonEmptyPartitions = this.partitions
.map(_.asInstanceOf[KafkaRDDPartition])
- .filter(_.count > 0)
+ .filter(_.count() > 0)
if (nonEmptyPartitions.isEmpty) {
Array.empty[ConsumerRecord[K, V]]
@@ -125,7 +125,7 @@
val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) =>
val remain = num - result.values.sum
if (remain > 0) {
- val taken = Math.min(remain, part.count)
+ val taken = Math.min(remain, part.count())
result + (part.index -> taken.toInt)
} else {
result
@@ -135,7 +135,7 @@
context.runJob(
this,
(tc: TaskContext, it: Iterator[ConsumerRecord[K, V]]) =>
- it.take(parts(tc.partitionId)).toArray, parts.keys.toArray
+ it.take(parts(tc.partitionId())).toArray, parts.keys.toArray
).flatten
}
}
@@ -162,7 +162,7 @@
// TODO what about hosts specified by ip vs name
val part = thePart.asInstanceOf[KafkaRDDPartition]
val allExecs = executors()
- val tp = part.topicPartition
+ val tp = part.topicPartition()
val prefHost = preferredHosts.get(tp)
val prefExecs = if (null == prefHost) allExecs else allExecs.filter(_.host == prefHost)
val execs = if (prefExecs.isEmpty) allExecs else prefExecs
diff --git a/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala
index 978baaa..7b2cac4a 100644
--- a/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala
+++ b/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala
@@ -90,7 +90,7 @@
kp.put("bootstrap.servers", kafkaTestUtils.brokerAddress)
kp.put("key.deserializer", classOf[StringDeserializer])
kp.put("value.deserializer", classOf[StringDeserializer])
- kp.put("group.id", s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}")
+ kp.put("group.id", s"test-consumer-${Random.nextInt()}-${System.currentTimeMillis}")
extra.foreach(e => kp.put(e._1, e._2))
kp
}
@@ -138,7 +138,7 @@
val partSize = all.size
val rangeSize = off.untilOffset - off.fromOffset
Iterator((partSize, rangeSize))
- }.collect
+ }.collect()
// Verify whether number of elements in each partition
// matches with the corresponding offset range
@@ -204,7 +204,7 @@
val partSize = all.size
val rangeSize = off.untilOffset - off.fromOffset
Iterator((partSize, rangeSize))
- }.collect
+ }.collect()
// Verify whether number of elements in each partition
// matches with the corresponding offset range
@@ -255,9 +255,9 @@
preferredHosts,
ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala),
new DefaultPerPartitionConfig(sparkConf))
- s.consumer.poll(0)
+ s.consumer().poll(0)
assert(
- s.consumer.position(topicPartition) >= offsetBeforeStart,
+ s.consumer().position(topicPartition) >= offsetBeforeStart,
"Start offset not from latest"
)
s
@@ -311,9 +311,9 @@
kafkaParams.asScala,
Map(topicPartition -> 11L)),
new DefaultPerPartitionConfig(sparkConf))
- s.consumer.poll(0)
+ s.consumer().poll(0)
assert(
- s.consumer.position(topicPartition) >= offsetBeforeStart,
+ s.consumer().position(topicPartition) >= offsetBeforeStart,
"Start offset not from latest"
)
s
diff --git a/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala b/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala
index 0d1e13b..d7591a9 100644
--- a/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala
+++ b/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala
@@ -123,8 +123,8 @@
@volatile var error: Throwable = null
def consume(i: Int): Unit = {
- val useCache = Random.nextBoolean
- val taskContext = if (Random.nextBoolean) {
+ val useCache = Random.nextBoolean()
+ val taskContext = if (Random.nextBoolean()) {
new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), 1, null, null, null)
} else {
null
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 986943ac6..2591fff 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
@@ -59,7 +59,7 @@
try {
try {
if (sc != null) {
- sc.stop
+ sc.stop()
sc = null
}
} finally {
@@ -77,7 +77,7 @@
"bootstrap.servers" -> kafkaTestUtils.brokerAddress,
"key.deserializer" -> classOf[StringDeserializer],
"value.deserializer" -> classOf[StringDeserializer],
- "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}"
+ "group.id" -> s"test-consumer-${Random.nextInt()}-${System.currentTimeMillis}"
).asJava
private val preferredHosts = LocationStrategies.PreferConsistent
@@ -130,7 +130,7 @@
test("basic usage") {
- val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}"
+ val topic = s"topicbasic-${Random.nextInt()}-${System.currentTimeMillis}"
kafkaTestUtils.createTopic(topic)
val messages = Array("the", "quick", "brown", "fox")
kafkaTestUtils.sendMessages(topic, messages)
@@ -142,13 +142,13 @@
val rdd = KafkaUtils.createRDD[String, String](sc, kafkaParams, offsetRanges, preferredHosts)
.map(_.value)
- val received = rdd.collect.toSet
+ val received = rdd.collect().toSet
assert(received === messages.toSet)
// size-related method optimizations return sane results
- assert(rdd.count === messages.size)
- assert(rdd.countApprox(0).getFinalValue.mean === messages.size)
- assert(!rdd.isEmpty)
+ assert(rdd.count() === messages.size)
+ assert(rdd.countApprox(0).getFinalValue().mean === messages.size)
+ assert(!rdd.isEmpty())
assert(rdd.take(1).size === 1)
assert(rdd.take(1).head === messages.head)
assert(rdd.take(messages.size + 10).size === messages.size)
@@ -156,7 +156,7 @@
val emptyRdd = KafkaUtils.createRDD[String, String](
sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0)), preferredHosts)
- assert(emptyRdd.isEmpty)
+ assert(emptyRdd.isEmpty())
// invalid offset ranges throw exceptions
val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1))
@@ -172,7 +172,7 @@
compactConf.set("spark.streaming.kafka.allowNonConsecutiveOffsets", "true")
sc.stop()
sc = new SparkContext(compactConf)
- val topic = s"topiccompacted-${Random.nextInt}-${System.currentTimeMillis}"
+ val topic = s"topiccompacted-${Random.nextInt()}-${System.currentTimeMillis}"
val messages = Array(
("a", "1"),
@@ -212,13 +212,13 @@
val dir = new File(kafkaTestUtils.brokerLogDir, topic + "-0")
assert(dir.listFiles().exists(_.getName.endsWith(".deleted")))
}
- val received = rdd.collect.toSet
+ val received = rdd.collect().toSet
assert(received === compactedMessages.toSet)
// size-related method optimizations return sane results
- assert(rdd.count === compactedMessages.size)
- assert(rdd.countApprox(0).getFinalValue.mean === compactedMessages.size)
- assert(!rdd.isEmpty)
+ assert(rdd.count() === compactedMessages.size)
+ assert(rdd.countApprox(0).getFinalValue().mean === compactedMessages.size)
+ assert(!rdd.isEmpty())
assert(rdd.take(1).size === 1)
assert(rdd.take(1).head === compactedMessages.head)
assert(rdd.take(messages.size + 10).size === compactedMessages.size)
@@ -226,7 +226,7 @@
val emptyRdd = KafkaUtils.createRDD[String, String](
sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0)), preferredHosts)
- assert(emptyRdd.isEmpty)
+ assert(emptyRdd.isEmpty())
// invalid offset ranges throw exceptions
val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1))
@@ -239,7 +239,7 @@
test("iterator boundary conditions") {
// the idea is to find e.g. off-by-one errors between what kafka has available and the rdd
- val topic = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}"
+ val topic = s"topicboundary-${Random.nextInt()}-${System.currentTimeMillis}"
val sent = Map("a" -> 5, "b" -> 3, "c" -> 10)
kafkaTestUtils.createTopic(topic)
@@ -256,7 +256,7 @@
val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum
assert(rangeCount === sentCount, "offset range didn't include all sent messages")
- assert(rdd.map(_.offset).collect.sorted === (0 until sentCount).toArray,
+ assert(rdd.map(_.offset).collect().sorted === (0 until sentCount).toArray,
"didn't get all sent messages")
// this is the "0 messages" case
@@ -268,7 +268,7 @@
kafkaTestUtils.sendMessages(topic, sentOnlyOne)
- assert(rdd2.map(_.value).collect.size === 0, "got messages when there shouldn't be any")
+ assert(rdd2.map(_.value).collect().size === 0, "got messages when there shouldn't be any")
// this is the "exactly 1 message" case, namely the single message from sentOnlyOne above
val rdd3 = KafkaUtils.createRDD[String, String](sc, kafkaParams,
@@ -277,7 +277,7 @@
// send lots of messages after rdd was defined, they shouldn't show up
kafkaTestUtils.sendMessages(topic, Map("extra" -> 22))
- assert(rdd3.map(_.value).collect.head === sentOnlyOne.keys.head,
+ assert(rdd3.map(_.value).collect().head === sentOnlyOne.keys.head,
"didn't get exactly one message")
}
diff --git a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala
index c76eb7c..f95bee1 100644
--- a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala
+++ b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala
@@ -69,7 +69,7 @@
sentSeqNumbers += ((num, seqNumber))
}
}
- Futures.addCallback(future, kinesisCallBack, ThreadUtils.sameThreadExecutorService)
+ Futures.addCallback(future, kinesisCallBack, ThreadUtils.sameThreadExecutorService())
}
producer.flushSync()
shardIdToSeqNumbers.mapValues(_.toSeq).toMap
diff --git a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala
index ba04c01..9f2e34e 100644
--- a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala
+++ b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala
@@ -184,7 +184,7 @@
.checkpointAppName(appName)
.checkpointInterval(checkpointInterval)
.storageLevel(storageLevel)
- .build
+ .build()
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala
index 8ffccdf..9bc7ade 100644
--- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala
+++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala
@@ -57,7 +57,7 @@
private val listener = new SparkListener {
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
val stageInfo = stageCompleted.stageInfo
- val barrierId = ContextBarrierId(stageInfo.stageId, stageInfo.attemptNumber)
+ val barrierId = ContextBarrierId(stageInfo.stageId, stageInfo.attemptNumber())
// Clear ContextBarrierState from a finished stage attempt.
cleanupBarrierStage(barrierId)
}
diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
index ff14103..0f9abaf 100644
--- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
@@ -56,14 +56,15 @@
private var barrierEpoch = 0
private def runBarrier(message: String, requestMethod: RequestMethod.Value): Array[String] = {
- logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) has entered " +
- s"the global sync, current barrier epoch is $barrierEpoch.")
+ logInfo(s"Task ${taskAttemptId()} from Stage ${stageId()}(Attempt ${stageAttemptNumber()}) " +
+ s"has entered the global sync, current barrier epoch is $barrierEpoch.")
logTrace("Current callSite: " + Utils.getCallSite())
val startTime = System.currentTimeMillis()
val timerTask = new TimerTask {
override def run(): Unit = {
- logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) waiting " +
+ logInfo(s"Task ${taskAttemptId()} from Stage ${stageId()}(Attempt " +
+ s"${stageAttemptNumber()}) waiting " +
s"under the global sync since $startTime, has been waiting for " +
s"${MILLISECONDS.toSeconds(System.currentTimeMillis() - startTime)} seconds, " +
s"current barrier epoch is $barrierEpoch.")
@@ -74,8 +75,8 @@
try {
val abortableRpcFuture = barrierCoordinator.askAbortable[Array[String]](
- message = RequestToSync(numPartitions, stageId, stageAttemptNumber, taskAttemptId,
- barrierEpoch, partitionId, message, requestMethod),
+ message = RequestToSync(numPartitions(), stageId(), stageAttemptNumber(), taskAttemptId(),
+ barrierEpoch, partitionId(), message, requestMethod),
// Set a fixed timeout for RPC here, so users shall get a SparkException thrown by
// BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework.
timeout = new RpcTimeout(365.days, "barrierTimeout"))
@@ -102,15 +103,15 @@
val messages = abortableRpcFuture.future.value.get.get
barrierEpoch += 1
- logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) finished " +
- "global sync successfully, waited for " +
+ logInfo(s"Task ${taskAttemptId()} from Stage ${stageId()}(Attempt ${stageAttemptNumber()}) " +
+ s"finished global sync successfully, waited for " +
s"${MILLISECONDS.toSeconds(System.currentTimeMillis() - startTime)} seconds, " +
s"current barrier epoch is $barrierEpoch.")
messages
} catch {
case e: SparkException =>
- logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) failed " +
- "to perform global sync, waited for " +
+ logInfo(s"Task ${taskAttemptId()} from Stage ${stageId()}(Attempt " +
+ s"${stageAttemptNumber()}) failed to perform global sync, waited for " +
s"${MILLISECONDS.toSeconds(System.currentTimeMillis() - startTime)} seconds, " +
s"current barrier epoch is $barrierEpoch.")
throw e
diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala
index 4f749dd..090458e 100644
--- a/core/src/main/scala/org/apache/spark/Heartbeater.scala
+++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala
@@ -40,7 +40,7 @@
/** Schedules a task to report a heartbeat. */
def start(): Unit = {
// Wait a random interval so the heartbeats don't end up in sync
- val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int]
+ val initialDelay = intervalMs + (math.random() * intervalMs).asInstanceOf[Int]
val heartbeatTask = new Runnable() {
override def run(): Unit = Utils.logUncaughtExceptions(reportHeartbeat())
diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala
index 821577f0..f8961ff 100644
--- a/core/src/main/scala/org/apache/spark/SecurityManager.scala
+++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala
@@ -304,7 +304,7 @@
* @return the secret key as a String if authentication is enabled, otherwise returns null
*/
def getSecretKey(): String = {
- if (isAuthenticationEnabled) {
+ if (isAuthenticationEnabled()) {
val creds = UserGroupInformation.getCurrentUser().getCredentials()
Option(creds.getSecretKey(SECRET_LOOKUP_KEY))
.map { bytes => new String(bytes, UTF_8) }
@@ -396,7 +396,7 @@
aclUsers: Set[String],
aclGroups: Set[String]): Boolean = {
if (user == null ||
- !aclsEnabled ||
+ !aclsEnabled() ||
aclUsers.contains(WILDCARD_ACL) ||
aclUsers.contains(user) ||
aclGroups.contains(WILDCARD_ACL)) {
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 008b963..893895e 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1688,7 +1688,7 @@
require(!classOf[RDD[_]].isAssignableFrom(classTag[T].runtimeClass),
"Can not directly broadcast RDDs; instead, call collect() and broadcast the result.")
val bc = env.broadcastManager.newBroadcast[T](value, isLocal, serializedOnly)
- val callSite = getCallSite
+ val callSite = getCallSite()
logInfo("Created broadcast " + bc.id + " from " + callSite.shortForm)
cleaner.foreach(_.registerBroadcastForCleanup(bc))
bc
@@ -2417,7 +2417,7 @@
if (stopped.get()) {
throw new IllegalStateException("SparkContext has been shutdown")
}
- val callSite = getCallSite
+ val callSite = getCallSite()
val cleanedFunc = clean(func)
logInfo("Starting job: " + callSite.shortForm)
if (conf.getBoolean("spark.logLineage", false)) {
@@ -2539,7 +2539,7 @@
evaluator: ApproximateEvaluator[U, R],
timeout: Long): PartialResult[R] = {
assertNotStopped()
- val callSite = getCallSite
+ val callSite = getCallSite()
logInfo("Starting job: " + callSite.shortForm)
val start = System.nanoTime
val cleanedFunc = clean(func)
@@ -2569,7 +2569,7 @@
{
assertNotStopped()
val cleanF = clean(processPartition)
- val callSite = getCallSite
+ val callSite = getCallSite()
val waiter = dagScheduler.submitJob(
rdd,
(context: TaskContext, iter: Iterator[T]) => cleanF(iter),
@@ -2733,7 +2733,7 @@
/** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */
def defaultParallelism: Int = {
assertNotStopped()
- taskScheduler.defaultParallelism
+ taskScheduler.defaultParallelism()
}
/**
@@ -2803,7 +2803,7 @@
val addedArchivePaths = allAddedArchives.keys.toSeq
val environmentDetails = SparkEnv.environmentDetails(conf, hadoopConfiguration,
schedulingMode, addedJarPaths, addedFilePaths, addedArchivePaths,
- env.metricsSystem.metricsProperties.asScala.toMap)
+ env.metricsSystem.metricsProperties().asScala.toMap)
val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails)
listenerBus.post(environmentUpdate)
}
diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
index f0dbfa9..8c3af98 100644
--- a/core/src/main/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -398,7 +398,7 @@
def withHttpServer(resBaseDir: String = ".")(body: URL => Unit): Unit = {
// 0 as port means choosing randomly from the available ports
- val server = new Server(new InetSocketAddress(Utils.localCanonicalHostName, 0))
+ val server = new Server(new InetSocketAddress(Utils.localCanonicalHostName(), 0))
val resHandler = new ResourceHandler()
resHandler.setResourceBase(resBaseDir)
val handlers = new HandlerList()
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 80c6e1f..6ead369 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -749,7 +749,7 @@
*
* @since 3.5.0
*/
- def getJobTags(): util.Set[String] = sc.getJobTags.asJava
+ def getJobTags(): util.Set[String] = sc.getJobTags().asJava
/**
* Clear the current thread's job tags.
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
index 4eafbbd..840352e 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
@@ -221,7 +221,7 @@
}
if (reuseWorker) {
- val key = (worker, context.taskAttemptId)
+ val key = (worker, context.taskAttemptId())
// SPARK-35009: avoid creating multiple monitor threads for the same python worker
// and task context
if (PythonRunner.runningMonitorThreads.add(key)) {
@@ -399,7 +399,7 @@
dataOut.flush()
} catch {
case t: Throwable if NonFatal(t) || t.isInstanceOf[Exception] =>
- if (context.isCompleted || context.isInterrupted) {
+ if (context.isCompleted() || context.isInterrupted()) {
logDebug("Exception/NonFatal Error thrown after task completion (likely due to " +
"cleanup)", t)
if (worker.channel.isConnected) {
@@ -507,8 +507,8 @@
init, finish))
val memoryBytesSpilled = stream.readLong()
val diskBytesSpilled = stream.readLong()
- context.taskMetrics.incMemoryBytesSpilled(memoryBytesSpilled)
- context.taskMetrics.incDiskBytesSpilled(diskBytesSpilled)
+ context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled)
+ context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled)
}
protected def handlePythonException(): PythonException = {
@@ -533,7 +533,7 @@
}
protected val handleException: PartialFunction[Throwable, OUT] = {
- case e: Exception if context.isInterrupted =>
+ case e: Exception if context.isInterrupted() =>
logDebug("Exception thrown after task interruption", e)
throw new TaskKilledException(context.getKillReason().getOrElse("unknown reason"))
@@ -570,16 +570,16 @@
private def monitorWorker(): Unit = {
// Kill the worker if it is interrupted, checking until task completion.
// TODO: This has a race condition if interruption occurs, as completed may still become true.
- while (!context.isInterrupted && !context.isCompleted) {
+ while (!context.isInterrupted() && !context.isCompleted()) {
Thread.sleep(2000)
}
- if (!context.isCompleted) {
+ if (!context.isCompleted()) {
Thread.sleep(taskKillTimeout)
- if (!context.isCompleted) {
+ if (!context.isCompleted()) {
try {
// Mimic the task name used in `Executor` to help the user find out the task to blame.
- val taskName = s"${context.partitionId}.${context.attemptNumber} " +
- s"in stage ${context.stageId} (TID ${context.taskAttemptId})"
+ val taskName = s"${context.partitionId()}.${context.attemptNumber()} " +
+ s"in stage ${context.stageId()} (TID ${context.taskAttemptId()})"
logWarning(s"Incomplete task $taskName interrupted: Attempting to kill Python Worker")
env.destroyPythonWorker(
pythonExec, workerModule, daemonModule, envVars.asScala.toMap, worker)
@@ -596,7 +596,7 @@
monitorWorker()
} finally {
if (reuseWorker) {
- val key = (worker, context.taskAttemptId)
+ val key = (worker, context.taskAttemptId())
PythonRunner.runningMonitorThreads.remove(key)
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala
index 0f93873..09add64 100644
--- a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala
@@ -236,7 +236,7 @@
val lines = new Array[String](errBufferSize)
var lineIdx = 0
override def run(): Unit = {
- for (line <- Source.fromInputStream(in).getLines) {
+ for (line <- Source.fromInputStream(in).getLines()) {
synchronized {
lines(lineIdx) = line
lineIdx = (lineIdx + 1) % errBufferSize
diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index 0a5399a..8c35608 100644
--- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -29,7 +29,7 @@
private def writeResourcesInfo(info: Map[String, ResourceInformation]): JObject = {
val jsonFields = info.map {
- case (k, v) => JField(k, v.toJson)
+ case (k, v) => JField(k, v.toJson())
}
JObject(jsonFields.toList)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index c100e92..bd6507c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -612,7 +612,7 @@
stream.flush()
// Get the output and discard any unnecessary lines from it.
- Source.fromString(new String(out.toByteArray(), StandardCharsets.UTF_8)).getLines
+ Source.fromString(new String(out.toByteArray(), StandardCharsets.UTF_8)).getLines()
.filter { line =>
!line.startsWith("log4j") && !line.startsWith("usage")
}
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 99b3184..3592673 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
@@ -203,7 +203,7 @@
if (!Utils.isTesting) {
ThreadUtils.newDaemonFixedThreadPool(NUM_PROCESSING_THREADS, "log-replay-executor")
} else {
- ThreadUtils.sameThreadExecutorService
+ ThreadUtils.sameThreadExecutorService()
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
index f2cd5b7..887adc4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
@@ -72,7 +72,7 @@
} else if (eventLogsUnderProcessCount > 0) {
<h4>No completed applications found!</h4>
} else {
- <h4>No completed applications found!</h4> ++ parent.emptyListingHtml
+ <h4>No completed applications found!</h4> ++ parent.emptyListingHtml()
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
index d689eb0..c04214d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -87,8 +87,8 @@
}
// set auth secret to env variable if needed
- if (securityMgr.isAuthenticationEnabled) {
- newEnvironment += (SecurityManager.ENV_AUTH_SECRET -> securityMgr.getSecretKey)
+ if (securityMgr.isAuthenticationEnabled()) {
+ newEnvironment += (SecurityManager.ENV_AUTH_SECRET -> securityMgr.getSecretKey())
}
// set SSL env variables if needed
newEnvironment ++= securityMgr.getEnvironmentForSslRpcPasswords
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 9fb66fa..44082ae 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -103,7 +103,7 @@
private val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500
private val REGISTRATION_RETRY_FUZZ_MULTIPLIER = {
val randomNumberGenerator = new Random(UUID.randomUUID.getMostSignificantBits)
- randomNumberGenerator.nextDouble + FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND
+ randomNumberGenerator.nextDouble() + FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND
}
private val INITIAL_REGISTRATION_RETRY_INTERVAL_SECONDS = (math.round(10 *
REGISTRATION_RETRY_FUZZ_MULTIPLIER))
diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala
index 5448d7d..463552f 100644
--- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala
@@ -174,7 +174,7 @@
val f = new File(pidDir, procfsStatFile)
new BufferedReader(new InputStreamReader(new FileInputStream(f), UTF_8))
}
- Utils.tryWithResource(openReader) { in =>
+ Utils.tryWithResource(openReader()) { in =>
val procInfo = in.readLine
val procInfoSplit = procInfo.split(" ")
val vmem = procInfoSplit(22).toLong
@@ -210,7 +210,7 @@
if (!isAvailable) {
return ProcfsMetrics(0, 0, 0, 0, 0, 0)
}
- val pids = computeProcessTree
+ val pids = computeProcessTree()
var allMetrics = ProcfsMetrics(0, 0, 0, 0, 0, 0)
for (p <- pids) {
try {
diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
index 95ac877..f0d6cba 100644
--- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
+++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
@@ -100,7 +100,7 @@
if (!processed) {
val fileIn = new PortableDataStream(split, context, index)
value = parseStream(fileIn)
- key = fileIn.getPath
+ key = fileIn.getPath()
processed = true
true
} else {
diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala
index 4eeec63..0643ab3 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala
@@ -83,14 +83,14 @@
val ret = sparkContext.runJob(rdd, (context: TaskContext, iter: Iterator[(K, V)]) => {
// SPARK-24552: Generate a unique "attempt ID" based on the stage and task attempt numbers.
// Assumes that there won't be more than Short.MaxValue attempts, at least not concurrently.
- val attemptId = (context.stageAttemptNumber << 16) | context.attemptNumber
+ val attemptId = (context.stageAttemptNumber() << 16) | context.attemptNumber()
executeTask(
context = context,
config = config,
jobTrackerId = jobTrackerId,
commitJobId = commitJobId,
- sparkPartitionId = context.partitionId,
+ sparkPartitionId = context.partitionId(),
sparkAttemptNumber = attemptId,
committer = committer,
iterator = iter)
diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
index 77bbbd9..c70087d 100644
--- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
@@ -67,13 +67,13 @@
}
def setAppId(appId: String): Unit = {
- if (connection != null && isConnected) {
+ if (connection != null && isConnected()) {
connection.send(new SetAppId(appId))
}
}
def setState(state: SparkAppHandle.State): Unit = {
- if (connection != null && isConnected && lastState != state) {
+ if (connection != null && isConnected() && lastState != state) {
connection.send(new SetState(state))
lastState = state
}
diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala
index 7f12789..73805c1 100644
--- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala
+++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala
@@ -145,7 +145,7 @@
}
executionPool.acquireMemory(
- numBytes, taskAttemptId, maybeGrowExecutionPool, () => computeMaxExecutionPoolSize)
+ numBytes, taskAttemptId, maybeGrowExecutionPool, () => computeMaxExecutionPoolSize())
}
override def acquireStorageMemory(
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index 2c4a228..777bc0a 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -101,12 +101,12 @@
registerSources()
}
registerSinks()
- sinks.foreach(_.start)
+ sinks.foreach(_.start())
}
def stop(): Unit = {
if (running) {
- sinks.foreach(_.stop)
+ sinks.foreach(_.stop())
registry.removeMatching((_: String, _: Metric) => true)
} else {
logWarning("Stopping a MetricsSystem that is not running")
diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
index 9f89c82..612a3da 100644
--- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -66,7 +66,7 @@
* Returns a future for retrieving the first num elements of the RDD.
*/
def takeAsync(num: Int): FutureAction[Seq[T]] = self.withScope {
- val callSite = self.context.getCallSite
+ val callSite = self.context.getCallSite()
val localProperties = self.context.getLocalProperties
// Cached thread pool to handle aggregation of subtasks.
implicit val executionContext = AsyncRDDActions.futureExecutionContext
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 0b5f6a3..9430a69 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -302,7 +302,7 @@
private val inputFormat = getInputFormat(jobConf)
HadoopRDD.addLocalConfiguration(
new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(createTime),
- context.stageId, theSplit.index, context.attemptNumber, jobConf)
+ context.stageId(), theSplit.index, context.attemptNumber(), jobConf)
reader =
try {
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 4f3683e..c4a35c4 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -118,7 +118,7 @@
override def run(): Unit = {
val err = proc.getErrorStream
try {
- for (line <- Source.fromInputStream(err)(encoding).getLines) {
+ for (line <- Source.fromInputStream(err)(encoding).getLines()) {
// scalastyle:off println
System.err.println(line)
// scalastyle:on println
@@ -182,7 +182,7 @@
}
// Return an iterator that read lines from the process's stdout
- val lines = Source.fromInputStream(proc.getInputStream)(encoding).getLines
+ val lines = Source.fromInputStream(proc.getInputStream)(encoding).getLines()
new Iterator[String] {
def next(): String = {
if (!hasNext) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
index 7339eb6..863bcd5 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
@@ -247,7 +247,7 @@
private def writePartitionerToCheckpointDir(
sc: SparkContext, partitioner: Partitioner, checkpointDirPath: Path): Unit = {
try {
- val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName)
+ val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName())
val bufferSize = sc.conf.get(BUFFER_SIZE)
val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration)
val fileOutputStream = fs.create(partitionerFilePath, false, bufferSize)
@@ -276,14 +276,14 @@
checkpointDirPath: String): Option[Partitioner] = {
try {
val bufferSize = sc.conf.get(BUFFER_SIZE)
- val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName)
+ val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName())
val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration)
val fileInputStream = fs.open(partitionerFilePath, bufferSize)
val serializer = SparkEnv.get.serializer.newInstance()
val partitioner = Utils.tryWithSafeFinally {
val deserializeStream = serializer.deserializeStream(fileInputStream)
Utils.tryWithSafeFinally {
- deserializeStream.readObject[Partitioner]
+ deserializeStream.readObject[Partitioner]()
} {
deserializeStream.close()
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 230e49db..7241376 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -701,7 +701,7 @@
val toVisit = waitingForVisit.remove(0)
if (!visited(toVisit)) {
visited += toVisit
- Option(toVisit.getResourceProfile).foreach(resourceProfiles += _)
+ Option(toVisit.getResourceProfile()).foreach(resourceProfiles += _)
toVisit.dependencies.foreach {
case shuffleDep: ShuffleDependency[_, _, _] =>
parents += shuffleDep
@@ -1605,7 +1605,7 @@
val locs = taskIdToLocations(id)
val part = partitions(id)
stage.pendingPartitions += id
- new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber, taskBinary,
+ new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber(), taskBinary,
part, stage.numPartitions, locs, artifacts, properties, serializedTaskMetrics,
Option(jobId), Option(sc.applicationId), sc.applicationAttemptId,
stage.rdd.isBarrier())
@@ -1616,7 +1616,7 @@
val p: Int = stage.partitions(id)
val part = partitions(p)
val locs = taskIdToLocations(id)
- new ResultTask(stage.id, stage.latestInfo.attemptNumber,
+ new ResultTask(stage.id, stage.latestInfo.attemptNumber(),
taskBinary, part, stage.numPartitions, locs, id, artifacts, properties,
serializedTaskMetrics, Option(jobId), Option(sc.applicationId),
sc.applicationAttemptId, stage.rdd.isBarrier())
@@ -1638,7 +1638,7 @@
}
taskScheduler.submitTasks(new TaskSet(
- tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties,
+ tasks.toArray, stage.id, stage.latestInfo.attemptNumber(), jobId, properties,
stage.resourceProfileId, shuffleId))
} else {
// Because we posted SparkListenerStageSubmitted earlier, we should mark
@@ -1939,10 +1939,10 @@
val failedStage = stageIdToStage(task.stageId)
val mapStage = shuffleIdToMapStage(shuffleId)
- if (failedStage.latestInfo.attemptNumber != task.stageAttemptId) {
+ if (failedStage.latestInfo.attemptNumber() != task.stageAttemptId) {
logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" +
s" ${task.stageAttemptId} and there is a more recent attempt for that stage " +
- s"(attempt ${failedStage.latestInfo.attemptNumber}) running")
+ s"(attempt ${failedStage.latestInfo.attemptNumber()}) running")
} else {
val ignoreStageFailure = ignoreDecommissionFetchFailure &&
isExecutorDecommissioningOrDecommissioned(taskScheduler, bmAddress)
@@ -2166,10 +2166,10 @@
// Always fail the current stage and retry all the tasks when a barrier task fail.
val failedStage = stageIdToStage(task.stageId)
- if (failedStage.latestInfo.attemptNumber != task.stageAttemptId) {
+ if (failedStage.latestInfo.attemptNumber() != task.stageAttemptId) {
logInfo(s"Ignoring task failure from $task as it's from $failedStage attempt" +
s" ${task.stageAttemptId} and there is a more recent attempt for that stage " +
- s"(attempt ${failedStage.latestInfo.attemptNumber}) running")
+ s"(attempt ${failedStage.latestInfo.attemptNumber()}) running")
} else {
logInfo(s"Marking $failedStage (${failedStage.name}) as failed due to a barrier task " +
"failed.")
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
index be88148..1f12b46 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
@@ -79,7 +79,7 @@
x => info.completionTime.getOrElse(System.currentTimeMillis()) - x
).getOrElse("-")
- s"Stage(${info.stageId}, ${info.attemptNumber}); Name: '${info.name}'; " +
+ s"Stage(${info.stageId}, ${info.attemptNumber()}); Name: '${info.name}'; " +
s"Status: ${info.getStatusString}$failureReason; numTasks: ${info.numTasks}; " +
s"Took: $timeTaken msec"
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index 39667ea..1ecd185 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -207,10 +207,10 @@
def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[AccumulatorV2[_, _]] = {
if (context != null) {
// Note: internal accumulators representing task metrics always count failed values
- context.taskMetrics.nonZeroInternalAccums() ++
+ context.taskMetrics().nonZeroInternalAccums() ++
// zero value external accumulators may still be useful, e.g. SQLMetrics, we should not
// filter them out.
- context.taskMetrics.externalAccums.filter(a => !taskFailed || a.countFailedValues)
+ context.taskMetrics().externalAccums.filter(a => !taskFailed || a.countFailedValues)
} else {
Seq.empty
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index fefca85..d00578e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -1214,10 +1214,10 @@
}
private def waitBackendReady(): Unit = {
- if (backend.isReady) {
+ if (backend.isReady()) {
return
}
- while (!backend.isReady) {
+ while (!backend.isReady()) {
// Might take a while for backend to be ready if it is waiting on resources.
if (sc.stopped.get) {
// For example: the master removes the application for some reason
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 de9c58e..6157a3e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -1152,7 +1152,7 @@
// config executorDecommissionKillInterval. If the task is going to finish after
// decommissioning, then we will eagerly speculate the task.
val taskEndTimeBasedOnMedianDuration =
- info.launchTime + successfulTaskDurations.percentile
+ info.launchTime + successfulTaskDurations.percentile()
val executorDecomTime = decomState.startTime + executorDecommissionKillInterval.get
executorDecomTime < taskEndTimeBasedOnMedianDuration
}
@@ -1195,7 +1195,7 @@
val numSuccessfulTasks = successfulTaskDurations.size()
val timeMs = clock.getTimeMillis()
if (numSuccessfulTasks >= minFinishedForSpeculation) {
- val medianDuration = successfulTaskDurations.percentile
+ val medianDuration = successfulTaskDurations.percentile()
val threshold = max(speculationMultiplier * medianDuration, minTimeToSpeculation)
// TODO: Threshold should also look at standard deviation of task durations and have a lower
// bound based on that.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 274f867..dd53757 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -248,7 +248,7 @@
attributes, resources, resourceProfileId) =>
if (executorDataMap.contains(executorId)) {
context.sendFailure(new IllegalStateException(s"Duplicate executor ID: $executorId"))
- } else if (scheduler.excludedNodes.contains(hostname) ||
+ } else if (scheduler.excludedNodes().contains(hostname) ||
isExecutorExcluded(executorId, hostname)) {
// If the cluster manager gives us an executor on an excluded node (because it
// already started allocating those resources before we informed it of our exclusion,
@@ -718,7 +718,7 @@
def sufficientResourcesRegistered(): Boolean = true
override def isReady(): Boolean = {
- if (sufficientResourcesRegistered) {
+ if (sufficientResourcesRegistered()) {
logInfo("SchedulerBackend is ready for scheduling beginning after " +
s"reached minRegisteredResourcesRatio: $minRegisteredRatio")
return true
@@ -881,7 +881,7 @@
// Note: it's possible that something else allocated an executor and we have
// a negative delta, we can just avoid mutating the queue.
while (toConsume > 0 && times.nonEmpty) {
- val h = times.dequeue
+ val h = times.dequeue()
if (h._1 > toConsume) {
// Prepend updated first req to times, constant time op
((h._1 - toConsume, h._2)) +=: times
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index f1be78a..482e186 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -217,7 +217,7 @@
override def applicationId(): String =
Option(appId).getOrElse {
logWarning("Application ID is not initialized yet.")
- super.applicationId
+ super.applicationId()
}
/**
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 60af1ab..6130808 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -168,11 +168,11 @@
// GenericDatum(Reader|Writer).
def registerAvro[T <: GenericContainer]()(implicit ct: ClassTag[T]): Unit =
kryo.register(ct.runtimeClass, new GenericAvroSerializer[T](avroSchemas))
- registerAvro[GenericRecord]
- registerAvro[GenericData.Record]
- registerAvro[GenericData.Array[_]]
- registerAvro[GenericData.EnumSymbol]
- registerAvro[GenericData.Fixed]
+ registerAvro[GenericRecord]()
+ registerAvro[GenericData.Record]()
+ registerAvro[GenericData.Array[_]]()
+ registerAvro[GenericData.EnumSymbol]()
+ registerAvro[GenericData.Fixed]()
// Use the default classloader when calling the user registrator.
Utils.withContextClassLoader(classLoader) {
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index 0ae053d..63c202a 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -250,7 +250,7 @@
}
// Remove all RDD partitions that reference the removed executor
liveRDDs.values.foreach { rdd =>
- rdd.getPartitions.values
+ rdd.getPartitions().values
.filter(_.executors.contains(event.executorId))
.foreach { partition =>
if (partition.executors.length == 1) {
@@ -812,7 +812,7 @@
override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
val maybeStage =
- Option(liveStages.get((event.stageInfo.stageId, event.stageInfo.attemptNumber)))
+ Option(liveStages.get((event.stageInfo.stageId, event.stageInfo.attemptNumber())))
maybeStage.foreach { stage =>
val now = System.nanoTime()
stage.info = event.stageInfo
@@ -860,7 +860,7 @@
val removeStage = stage.activeTasks == 0
update(stage, now, last = removeStage)
if (removeStage) {
- liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptNumber))
+ liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptNumber()))
}
if (stage.status == v1.StageStatus.COMPLETE) {
appSummary = new AppSummary(appSummary.numCompletedJobs, appSummary.numCompletedStages + 1)
@@ -1209,7 +1209,7 @@
}
private def getOrCreateStage(info: StageInfo): LiveStage = {
- val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber),
+ val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber()),
(_: (Int, Int)) => new LiveStage(info))
stage.info = info
stage
@@ -1377,7 +1377,7 @@
private def cleanupTasks(stage: LiveStage): Unit = {
val countToDelete = calculateNumberToRemove(stage.savedTasks.get(), maxTasksPerStage).toInt
if (countToDelete > 0) {
- val stageKey = Array(stage.info.stageId, stage.info.attemptNumber)
+ val stageKey = Array(stage.info.stageId, stage.info.attemptNumber())
val view = kvstore.view(classOf[TaskDataWrapper])
.index(TaskIndexNames.COMPLETION_TIME)
.parent(stageKey)
diff --git a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
index 2bc8f4d..4aef7f9 100644
--- a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
@@ -73,7 +73,7 @@
private val executor: ExecutorService = if (conf.get(ASYNC_TRACKING_ENABLED)) {
ThreadUtils.newDaemonSingleThreadExecutor("element-tracking-store-worker")
} else {
- ThreadUtils.sameThreadExecutorService
+ ThreadUtils.sameThreadExecutorService()
}
@volatile private var stopped = false
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index eb51cb9..efc6704 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -364,7 +364,7 @@
executorLogs,
memoryMetrics,
excludedInStages,
- Some(peakExecutorMetrics).filter(_.isSet),
+ Some(peakExecutorMetrics).filter(_.isSet()),
attributes,
resources,
resourceProfileId,
@@ -408,7 +408,7 @@
metrics.memoryBytesSpilled,
metrics.diskBytesSpilled,
isExcluded,
- Some(peakExecutorMetrics).filter(_.isSet),
+ Some(peakExecutorMetrics).filter(_.isSet()),
isExcluded)
new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info)
}
@@ -472,7 +472,7 @@
val peakExecutorMetrics = new ExecutorMetrics()
lazy val speculationStageSummary: LiveSpeculationStageSummary =
- new LiveSpeculationStageSummary(info.stageId, info.attemptNumber)
+ new LiveSpeculationStageSummary(info.stageId, info.attemptNumber())
// Used for cleanup of tasks after they reach the configured limit. Not written to the store.
@volatile var cleaning = false
@@ -480,14 +480,14 @@
def executorSummary(executorId: String): LiveExecutorStageSummary = {
executorSummaries.getOrElseUpdate(executorId,
- new LiveExecutorStageSummary(info.stageId, info.attemptNumber, executorId))
+ new LiveExecutorStageSummary(info.stageId, info.attemptNumber(), executorId))
}
def toApi(): v1.StageData = {
new v1.StageData(
status = status,
stageId = info.stageId,
- attemptId = info.attemptNumber,
+ attemptId = info.attemptNumber(),
numTasks = info.numTasks,
numActiveTasks = activeTasks,
numCompleteTasks = completedTasks,
@@ -559,7 +559,7 @@
speculationSummary = None,
killedTasksSummary = killedSummary,
resourceProfileId = info.resourceProfileId,
- peakExecutorMetrics = Some(peakExecutorMetrics).filter(_.isSet),
+ peakExecutorMetrics = Some(peakExecutorMetrics).filter(_.isSet()),
taskMetricsDistributions = None,
executorMetricsDistributions = None,
isShufflePushEnabled = info.isShufflePushEnabled,
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala
index ca088dc..8cfed4a 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala
@@ -48,8 +48,8 @@
store.executorList(true).foreach { executor =>
val prefix = "metrics_executor_"
val labels = Seq(
- "application_id" -> store.applicationInfo.id,
- "application_name" -> store.applicationInfo.name,
+ "application_id" -> store.applicationInfo().id,
+ "application_name" -> store.applicationInfo().name,
"executor_id" -> executor.id
).map { case (k, v) => s"""$k="$v"""" }.mkString("{", ", ", "}")
sb.append(s"${prefix}rddBlocks$labels ${executor.rddBlocks}\n")
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 454c770..aa9ba7c 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -1315,7 +1315,7 @@
val taskAttemptId = taskContext.map(_.taskAttemptId())
// SPARK-27666. When a task completes, Spark automatically releases all the blocks locked
// by this task. We should not release any locks for a task that is already completed.
- if (taskContext.isDefined && taskContext.get.isCompleted) {
+ if (taskContext.isDefined && taskContext.get.isCompleted()) {
logWarning(s"Task ${taskAttemptId.get} already completed, not releasing lock for $blockId")
} else {
blockInfoManager.unlock(blockId, taskAttemptId)
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index ca111a8..948acb7 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -67,30 +67,30 @@
def nextFloat(): Float = new Random().nextFloat()
val jobs = Seq[(String, () => Long)](
- ("Count", () => baseData.count),
- ("Cache and Count", () => baseData.map(x => x).cache().count),
- ("Single Shuffle", () => baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count),
- ("Entirely failed phase", () => baseData.map { x => throw new Exception(); 1 }.count),
+ ("Count", () => baseData.count()),
+ ("Cache and Count", () => baseData.map(x => x).cache().count()),
+ ("Single Shuffle", () => baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count()),
+ ("Entirely failed phase", () => baseData.map { x => throw new Exception(); 1 }.count()),
("Partially failed phase", () => {
- baseData.map{x =>
+ baseData.map { x =>
val probFailure = (4.0 / NUM_PARTITIONS)
if (nextFloat() < probFailure) {
throw new Exception("This is a task failure")
}
1
- }.count
+ }.count()
}),
("Partially failed phase (longer tasks)", () => {
- baseData.map{x =>
+ baseData.map { x =>
val probFailure = (4.0 / NUM_PARTITIONS)
if (nextFloat() < probFailure) {
Thread.sleep(100)
throw new Exception("This is a task failure")
}
1
- }.count
+ }.count()
}),
- ("Job with delays", () => baseData.map(x => Thread.sleep(100)).count)
+ ("Job with delays", () => baseData.map(x => Thread.sleep(100)).count())
)
val barrier = new Semaphore(-nJobSet * jobs.size + 1)
diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
index e531bcf..9a035e0 100644
--- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
+++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
@@ -136,7 +136,7 @@
// Use a special prefix here to differentiate this cluster from other operation clusters
val stageClusterId = STAGE_CLUSTER_PREFIX + stage.stageId
val stageClusterName = s"Stage ${stage.stageId}" +
- { if (stage.attemptNumber == 0) "" else s" (attempt ${stage.attemptNumber})" }
+ { if (stage.attemptNumber() == 0) "" else s" (attempt ${stage.attemptNumber()})" }
val rootCluster = new RDDOperationCluster(stageClusterId, false, stageClusterName)
var rootNodeCount = 0
diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
index 54c267c..29fb020 100644
--- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
@@ -685,7 +685,7 @@
}
while (!stack.isEmpty) {
- val currentId = stack.pop
+ val currentId = stack.pop()
visited += currentId
val currentClass = currentId.cls
diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index 6525bd3..b3c208b 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -387,7 +387,7 @@
includeAccumulables: Boolean): Unit = {
g.writeStartObject()
g.writeNumberField("Stage ID", stageInfo.stageId)
- g.writeNumberField("Stage Attempt ID", stageInfo.attemptNumber)
+ g.writeNumberField("Stage Attempt ID", stageInfo.attemptNumber())
g.writeStringField("Stage Name", stageInfo.name)
g.writeNumberField ("Number of Tasks", stageInfo.numTasks)
g.writeArrayFieldStart("RDD Info")
diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
index 10ff801..704aeae 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -197,7 +197,7 @@
private def estimate(obj: AnyRef, visited: IdentityHashMap[AnyRef, AnyRef]): Long = {
val state = new SearchState(visited)
state.enqueue(obj)
- while (!state.isFinished) {
+ while (!state.isFinished()) {
visitSingleObject(state.dequeue(), state)
}
state.size
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 71f3b0b..3afbe32 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
@@ -565,7 +565,7 @@
if (hasSpilled) {
false
} else {
- logInfo(s"Task ${context.taskAttemptId} force spilling in-memory map to disk and " +
+ logInfo(s"Task ${context.taskAttemptId()} force spilling in-memory map to disk and " +
s"it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory")
val nextUpstream = spillMemoryIteratorToDisk(upstream)
assert(!upstream.hasNext)
@@ -581,7 +581,7 @@
}
def toCompletionIterator: CompletionIterator[(K, C), SpillableIterator] = {
- CompletionIterator[(K, C), SpillableIterator](this, this.destroy)
+ CompletionIterator[(K, C), SpillableIterator](this, this.destroy())
}
def readNext(): (K, C) = SPILL_LOCK.synchronized {
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 4250172..5051b88 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
@@ -816,7 +816,7 @@
false
} else {
val inMemoryIterator = new WritablePartitionedIterator[K, C](upstream)
- logInfo(s"Task ${TaskContext.get().taskAttemptId} force spilling in-memory map to disk " +
+ logInfo(s"Task ${TaskContext.get().taskAttemptId()} force spilling in-memory map to disk " +
s"and it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory")
val spillFile = spillMemoryIteratorToDisk(inMemoryIterator)
forceSpillFiles += spillFile
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ImmutableBitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/ImmutableBitSet.scala
index 82413f4..8047f50 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ImmutableBitSet.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ImmutableBitSet.scala
@@ -28,7 +28,7 @@
{
val bitsIterator = bitsToSet.iterator
while (bitsIterator.hasNext) {
- super.set(bitsIterator.next)
+ super.set(bitsIterator.next())
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala
index 652d8c0..b43a8c7 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala
@@ -77,7 +77,7 @@
: Iterator[((Int, K), V)] = {
val comparator = keyComparator.map(partitionKeyComparator).getOrElse(partitionComparator)
new Sorter(new KVArraySortDataFormat[(Int, K), AnyRef]).sort(data, 0, curSize, comparator)
- iterator
+ iterator()
}
private def iterator(): Iterator[((Int, K), V)] = new Iterator[((Int, K), V)] {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PercentileHeap.scala b/core/src/main/scala/org/apache/spark/util/collection/PercentileHeap.scala
index ac6bc0e..61f95f8 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PercentileHeap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PercentileHeap.scala
@@ -47,16 +47,16 @@
* returned `sorted(p)` where `p = (sorted.length * percentage).toInt`.
*/
def percentile(): Double = {
- if (isEmpty) throw new NoSuchElementException("empty")
+ if (isEmpty()) throw new NoSuchElementException("empty")
largeHeap.peek
}
def insert(x: Double): Unit = {
- if (isEmpty) {
+ if (isEmpty()) {
largeHeap.offer(x)
} else {
val p = largeHeap.peek
- val growBot = ((size + 1) * percentage).toInt > smallHeap.size
+ val growBot = ((size() + 1) * percentage).toInt > smallHeap.size
if (growBot) {
if (x < p) {
smallHeap.offer(-x)
diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
index 6dd2bee..8bf2a70 100644
--- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
@@ -39,7 +39,7 @@
/** take a random sample */
def sample(items: Iterator[T]): Iterator[U] =
- items.filter(_ => sample > 0).asInstanceOf[Iterator[U]]
+ items.filter(_ => sample() > 0).asInstanceOf[Iterator[U]]
/**
* Whether to sample the next item or not.
diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
index 5434e82..b78bb1f 100644
--- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
@@ -89,7 +89,7 @@
val rdd: RDD[_] = Random.nextInt(3) match {
case 0 => newRDD()
case 1 => newShuffleRDD()
- case 2 => newPairRDD.join(newPairRDD())
+ case 2 => newPairRDD().join(newPairRDD())
}
if (Random.nextBoolean()) rdd.persist()
rdd.count()
@@ -288,7 +288,7 @@
val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer
val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast()).toBuffer
val rddIds = sc.persistentRdds.keys.toSeq
- val shuffleIds = 0 until sc.newShuffleId
+ val shuffleIds = 0 until sc.newShuffleId()
val broadcastIds = broadcastBuffer.map(_.id)
val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds.toSeq)
@@ -328,7 +328,7 @@
val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer
val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast()).toBuffer
val rddIds = sc.persistentRdds.keys.toSeq
- val shuffleIds = 0 until sc.newShuffleId
+ val shuffleIds = 0 until sc.newShuffleId()
val broadcastIds = broadcastBuffer.map(_.id)
val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds.toSeq)
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index ce1df3a..e156533 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -177,7 +177,7 @@
TestUtils.waitUntilExecutorsUp(sc, 3, 60000)
val data = sc.parallelize(1 to 1000, 10)
val cachedData = data.persist(storageLevel)
- assert(cachedData.count === 1000)
+ assert(cachedData.count() === 1000)
assert(sc.getRDDStorageInfo.filter(_.id == cachedData.id).map(_.numCachedPartitions).sum ===
data.getNumPartitions)
// Get all the locations of the first partition and try to fetch the partitions
@@ -263,9 +263,9 @@
DistributedSuite.amMaster = true
sc = new SparkContext(clusterUrl, "test")
val data = sc.parallelize(Seq(true, true), 2)
- assert(data.count === 2) // force executors to start
- assert(data.map(markNodeIfIdentity).collect.size === 2)
- assert(data.map(failOnMarkedIdentity).collect.size === 2)
+ assert(data.count() === 2) // force executors to start
+ assert(data.map(markNodeIfIdentity).collect().size === 2)
+ assert(data.map(failOnMarkedIdentity).collect().size === 2)
}
test("recover from repeated node failures during shuffle-map") {
@@ -274,9 +274,9 @@
sc = new SparkContext(clusterUrl, "test")
for (i <- 1 to 3) {
val data = sc.parallelize(Seq(true, false), 2)
- assert(data.count === 2)
- assert(data.map(markNodeIfIdentity).collect.size === 2)
- assert(data.map(failOnMarkedIdentity).map(x => x -> x).groupByKey.count === 2)
+ assert(data.count() === 2)
+ assert(data.map(markNodeIfIdentity).collect().size === 2)
+ assert(data.map(failOnMarkedIdentity).map(x => x -> x).groupByKey().count() === 2)
}
}
@@ -286,8 +286,8 @@
sc = new SparkContext(clusterUrl, "test")
for (i <- 1 to 3) {
val data = sc.parallelize(Seq(true, true), 2)
- assert(data.count === 2)
- assert(data.map(markNodeIfIdentity).collect.size === 2)
+ assert(data.count() === 2)
+ assert(data.map(markNodeIfIdentity).collect().size === 2)
// This relies on mergeCombiners being used to perform the actual reduce for this
// test to actually be testing what it claims.
val grouped = data.map(x => x -> x).combineByKey(
@@ -295,7 +295,7 @@
(x: Boolean, y: Boolean) => x,
(x: Boolean, y: Boolean) => failOnMarkedIdentity(x)
)
- assert(grouped.collect.size === 1)
+ assert(grouped.collect().size === 1)
}
}
@@ -309,14 +309,14 @@
val data = sc.parallelize(Seq(true, false, false, false), 4)
data.persist(StorageLevel.MEMORY_ONLY_2)
- assert(data.count === 4)
- assert(data.map(markNodeIfIdentity).collect.size === 4)
- assert(data.map(failOnMarkedIdentity).collect.size === 4)
+ assert(data.count() === 4)
+ assert(data.map(markNodeIfIdentity).collect().size === 4)
+ assert(data.map(failOnMarkedIdentity).collect().size === 4)
// Create a new replicated RDD to make sure that cached peer information doesn't cause
// problems.
val data2 = sc.parallelize(Seq(true, true), 2).persist(StorageLevel.MEMORY_ONLY_2)
- assert(data2.count === 2)
+ assert(data2.count() === 2)
}
}
@@ -325,7 +325,7 @@
sc = new SparkContext("local-cluster[3,1,1024]", "test")
val data = sc.parallelize(Seq(true, false, false, false), 4)
data.persist(StorageLevel.MEMORY_ONLY_2)
- data.count
+ data.count()
assert(sc.persistentRdds.nonEmpty)
data.unpersist(blocking = true)
assert(sc.persistentRdds.isEmpty)
diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
index 41d674f..e1da2b6 100644
--- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
@@ -180,7 +180,7 @@
val execReqs = new ExecutorResourceRequests().cores(4).resource("gpu", 4)
val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1)
rp1.require(execReqs).require(taskReqs)
- val rprof1 = rp1.build
+ val rprof1 = rp1.build()
rpManager.addResourceProfile(rprof1)
post(SparkListenerStageSubmitted(createStageInfo(1, 1000, rp = rprof1)))
val updatesNeeded =
@@ -277,7 +277,7 @@
val execReqs = new ExecutorResourceRequests().cores(2).resource("gpu", 2)
val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1)
rp1.require(execReqs).require(taskReqs)
- val rprof1 = rp1.build
+ val rprof1 = rp1.build()
rpManager.addResourceProfile(rprof1)
when(client.requestTotalExecutors(any(), any(), any())).thenReturn(true)
post(SparkListenerStageSubmitted(createStageInfo(1, 4, rp = rprof1)))
@@ -292,12 +292,12 @@
val execReqs = new ExecutorResourceRequests().cores(4).resource("gpu", 4)
val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1)
rp1.require(execReqs).require(taskReqs)
- val rprof1 = rp1.build
+ val rprof1 = rp1.build()
val rp2 = new ResourceProfileBuilder()
val execReqs2 = new ExecutorResourceRequests().cores(1)
val taskReqs2 = new TaskResourceRequests().cpus(1)
rp2.require(execReqs2).require(taskReqs2)
- val rprof2 = rp2.build
+ val rprof2 = rp2.build()
rpManager.addResourceProfile(rprof1)
rpManager.addResourceProfile(rprof2)
post(SparkListenerStageSubmitted(createStageInfo(1, 10, rp = rprof1)))
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index 64e3df7..8977b3b 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -86,11 +86,11 @@
data.saveAsTextFile(compressedOutputDir, classOf[DefaultCodec])
val normalFile = new File(normalDir, "part-00000")
- val normalContent = sc.textFile(normalDir).collect
+ val normalContent = sc.textFile(normalDir).collect()
assert(normalContent === Array.fill(10000)("a"))
val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension)
- val compressedContent = sc.textFile(compressedOutputDir).collect
+ val compressedContent = sc.textFile(compressedOutputDir).collect()
assert(compressedContent === Array.fill(10000)("a"))
assert(compressedFile.length < normalFile.length)
@@ -125,11 +125,11 @@
data.saveAsSequenceFile(compressedOutputDir, Some(codec.getClass))
val normalFile = new File(normalDir, "part-00000")
- val normalContent = sc.sequenceFile[String, String](normalDir).collect
+ val normalContent = sc.sequenceFile[String, String](normalDir).collect()
assert(normalContent === Array.fill(100)(("abc", "abc")))
val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension)
- val compressedContent = sc.sequenceFile[String, String](compressedOutputDir).collect
+ val compressedContent = sc.sequenceFile[String, String](compressedOutputDir).collect()
assert(compressedContent === Array.fill(100)(("abc", "abc")))
assert(compressedFile.length < normalFile.length)
@@ -288,7 +288,7 @@
val (infile, indata) = inRdd.collect().head
// Make sure the name and array match
assert(infile.contains(outFile.toURI.getPath)) // a prefix may get added
- assert(indata.toArray === testOutput)
+ assert(indata.toArray() === testOutput)
}
test("portabledatastream caching tests") {
@@ -298,7 +298,7 @@
val inRdd = sc.binaryFiles(outFile.getAbsolutePath).cache()
inRdd.foreach(_._2.toArray()) // force the file to read
// Try reading the output back as an object file
- assert(inRdd.values.collect().head.toArray === testOutput)
+ assert(inRdd.values.collect().head.toArray() === testOutput)
}
test("portabledatastream persist disk storage") {
@@ -307,7 +307,7 @@
val outFile = writeBinaryData(testOutput, 1)
val inRdd = sc.binaryFiles(outFile.getAbsolutePath).persist(StorageLevel.DISK_ONLY)
inRdd.foreach(_._2.toArray()) // force the file to read
- assert(inRdd.values.collect().head.toArray === testOutput)
+ assert(inRdd.values.collect().head.toArray() === testOutput)
}
test("portabledatastream flatmap tests") {
@@ -320,7 +320,7 @@
val copyArr = copyRdd.collect()
assert(copyArr.length == numOfCopies)
for (i <- copyArr.indices) {
- assert(copyArr(i).toArray === testOutput)
+ assert(copyArr(i).toArray() === testOutput)
}
}
@@ -354,7 +354,7 @@
"mapreduce.input.fileinputformat.split.minsize.per.rack", 5123456)
val (_, data) = sc.binaryFiles(outFile.getAbsolutePath).collect().head
- assert(data.toArray === testOutput)
+ assert(data.toArray() === testOutput)
}
test("fixed record length binary file as byte array") {
@@ -363,7 +363,7 @@
val testOutputCopies = 10
val outFile = writeBinaryData(testOutput, testOutputCopies)
val inRdd = sc.binaryRecords(outFile.getAbsolutePath, testOutput.length)
- assert(inRdd.count == testOutputCopies)
+ assert(inRdd.count() == testOutputCopies)
val inArr = inRdd.collect()
for (i <- inArr.indices) {
assert(inArr(i) === testOutput.map(b => (b + i).toByte))
@@ -699,7 +699,7 @@
intercept[org.apache.hadoop.mapreduce.lib.input.InvalidInputException] {
// Exception happens when NewHadoopRDD.getPartitions
sc.newAPIHadoopFile(deletedPath.toString, classOf[NewTextInputFormat],
- classOf[LongWritable], classOf[Text]).collect
+ classOf[LongWritable], classOf[Text]).collect()
}
e = intercept[SparkException] {
diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
index 95ef417..e3463a9 100644
--- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
@@ -77,7 +77,7 @@
sc = spy[SparkContext](new SparkContext(conf))
scheduler = mock(classOf[TaskSchedulerImpl])
when(sc.taskScheduler).thenReturn(scheduler)
- when(scheduler.excludedNodes).thenReturn(Predef.Set[String]())
+ when(scheduler.excludedNodes()).thenReturn(Predef.Set[String]())
when(scheduler.sc).thenReturn(sc)
heartbeatReceiverClock = new ManualClock
heartbeatReceiver = new HeartbeatReceiver(sc, heartbeatReceiverClock)
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
index fb2c44b..23225b2 100644
--- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -60,7 +60,7 @@
testCount()
testTake()
// Make sure we can still launch tasks.
- assert(sc.parallelize(1 to 10, 2).count === 10)
+ assert(sc.parallelize(1 to 10, 2).count() === 10)
}
test("local mode, fair scheduler") {
@@ -71,7 +71,7 @@
testCount()
testTake()
// Make sure we can still launch tasks.
- assert(sc.parallelize(1 to 10, 2).count === 10)
+ assert(sc.parallelize(1 to 10, 2).count() === 10)
}
test("cluster mode, FIFO scheduler") {
@@ -80,7 +80,7 @@
testCount()
testTake()
// Make sure we can still launch tasks.
- assert(sc.parallelize(1 to 10, 2).count === 10)
+ assert(sc.parallelize(1 to 10, 2).count() === 10)
}
test("cluster mode, fair scheduler") {
@@ -91,7 +91,7 @@
testCount()
testTake()
// Make sure we can still launch tasks.
- assert(sc.parallelize(1 to 10, 2).count === 10)
+ assert(sc.parallelize(1 to 10, 2).count() === 10)
}
test("do not put partially executed partitions into cache") {
diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
index 1a3259c..eea7753 100644
--- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
@@ -241,13 +241,13 @@
// Run the partitions, including the consecutive empty ones, through StatCounter
val stats: StatCounter = rdd.stats()
assert(abs(6.0 - stats.sum) < 0.01)
- assert(abs(6.0/2 - rdd.mean) < 0.01)
- assert(abs(1.0 - rdd.variance) < 0.01)
- assert(abs(1.0 - rdd.stdev) < 0.01)
- assert(abs(rdd.variance - rdd.popVariance) < 1e-14)
- assert(abs(rdd.stdev - rdd.popStdev) < 1e-14)
- assert(abs(2.0 - rdd.sampleVariance) < 1e-14)
- assert(abs(Math.sqrt(2.0) - rdd.sampleStdev) < 1e-14)
+ assert(abs(6.0/2 - rdd.mean()) < 0.01)
+ assert(abs(1.0 - rdd.variance()) < 0.01)
+ assert(abs(1.0 - rdd.stdev()) < 0.01)
+ assert(abs(rdd.variance() - rdd.popVariance()) < 1e-14)
+ assert(abs(rdd.stdev() - rdd.popStdev()) < 1e-14)
+ assert(abs(2.0 - rdd.sampleVariance()) < 1e-14)
+ assert(abs(Math.sqrt(2.0) - rdd.sampleStdev()) < 1e-14)
assert(stats.max === 4.0)
assert(stats.min === 2.0)
diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
index 0b6fce5..d403f94 100644
--- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
@@ -73,7 +73,7 @@
c.setSerializer(new KryoSerializer(conf))
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId
- assert(c.count === 10)
+ assert(c.count() === 10)
// All blocks must have non-zero size
(0 until NUM_BLOCKS).foreach { id =>
@@ -95,7 +95,7 @@
NonJavaSerializableClass,
NonJavaSerializableClass](b, new HashPartitioner(3))
c.setSerializer(new KryoSerializer(conf))
- assert(c.count === 10)
+ assert(c.count() === 10)
}
test("zero sized blocks") {
@@ -113,7 +113,7 @@
.setSerializer(new KryoSerializer(conf))
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId
- assert(c.count === 4)
+ assert(c.count() === 4)
val blockSizes = (0 until NUM_BLOCKS).flatMap { id =>
val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id)
@@ -138,7 +138,7 @@
val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(NUM_BLOCKS))
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId
- assert(c.count === 4)
+ assert(c.count() === 4)
val blockSizes = (0 until NUM_BLOCKS).flatMap { id =>
val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id)
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index 4145975..e140180 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -852,8 +852,8 @@
sc.addSparkListener(listener)
sc.range(0, 2).groupBy((x: Long) => x % 2, 2).map { case (x, _) =>
val context = org.apache.spark.TaskContext.get()
- if (context.stageAttemptNumber == 0) {
- if (context.partitionId == 0) {
+ if (context.stageAttemptNumber() == 0) {
+ if (context.partitionId() == 0) {
// Make the first task in the first stage attempt fail.
throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0L, 0, 0,
new java.io.IOException("fake"))
diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
index cd77d89..e3792eb 100644
--- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
@@ -200,7 +200,7 @@
protected def logForFailedTest(): Unit = {
LocalSparkCluster.get.foreach { localCluster =>
- val workerLogfiles = localCluster.workerLogfiles
+ val workerLogfiles = localCluster.workerLogfiles()
if (workerLogfiles.nonEmpty) {
logInfo("\n\n===== EXTRA LOGS FOR THE FAILED TEST\n")
workerLogfiles.foreach { logFile =>
diff --git a/core/src/test/scala/org/apache/spark/ThreadAudit.scala b/core/src/test/scala/org/apache/spark/ThreadAudit.scala
index 538cf3c9..a1c2d60 100644
--- a/core/src/test/scala/org/apache/spark/ThreadAudit.scala
+++ b/core/src/test/scala/org/apache/spark/ThreadAudit.scala
@@ -122,7 +122,7 @@
}
private def runningThreadNames(): Set[String] = {
- runningThreads.map(_.getName).toSet
+ runningThreads().map(_.getName)
}
private def runningThreads(): Set[Thread] = {
diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
index ff6ee79..fc70e13 100644
--- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
@@ -28,7 +28,7 @@
test("unpersist RDD") {
sc = new SparkContext("local", "test")
val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache()
- rdd.count
+ rdd.count()
assert(sc.persistentRdds.nonEmpty)
rdd.unpersist(blocking = true)
assert(sc.persistentRdds.isEmpty)
diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala
index 5511852..0b33e2a 100644
--- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala
+++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala
@@ -105,7 +105,7 @@
println(" Running case: " + c.name)
measure(valuesPerIteration, c.numIters)(c.fn)
}
- println
+ println()
val firstBest = results.head.bestMs
// The results are going to be processor specific so it is useful to include that.
@@ -125,7 +125,7 @@
"%6.1f" format (1000 / result.bestRate),
"%3.1fX" format (firstBest / result.bestMs))
}
- out.println
+ out.println()
// scalastyle:on
}
@@ -136,7 +136,7 @@
def measure(num: Long, overrideNumIters: Int)(f: Timer => Unit): Result = {
System.gc() // ensures garbage from previous cases don't impact this one
val warmupDeadline = warmupTime.fromNow
- while (!warmupDeadline.isOverdue) {
+ while (!warmupDeadline.isOverdue()) {
f(new Benchmark.Timer(-1))
}
val minIters = if (overrideNumIters != 0) overrideNumIters else minNumIters
diff --git a/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala
index 55d8535..b22c07d 100644
--- a/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala
@@ -213,7 +213,7 @@
TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
val executorIdToWorkerInfo = getExecutorToWorkerAssignments
- val executorToDecom = executorIdToWorkerInfo.keysIterator.next
+ val executorToDecom = executorIdToWorkerInfo.keysIterator.next()
// The task code below cannot call executorIdToWorkerInfo, so we need to pre-compute
// the worker to decom to force it to be serialized into the task.
@@ -249,7 +249,7 @@
}, preservesPartitioning = true)
.repartition(1).mapPartitions(iter => {
val context = TaskContext.get()
- if (context.attemptNumber == 0 && context.stageAttemptNumber() == 0) {
+ if (context.attemptNumber() == 0 && context.stageAttemptNumber() == 0) {
// Wait a bit for the decommissioning to be triggered in the listener
Thread.sleep(5000)
// MapIndex is explicitly -1 to force the entire host to be decommissioned
diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
index 5d60aad..98f475e 100644
--- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
@@ -59,7 +59,7 @@
sc.parallelize(1 to 100, 4).map(_.toString).count()
sc.listenerBus.waitUntilEmpty()
- val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo]
+ val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo]()
assert(listeners.size === 1)
val listener = listeners(0)
listener.addedExecutorInfos.values.foreach { info =>
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala
index c905797..c681093 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala
@@ -83,12 +83,12 @@
// Fail one of the tasks, re-start it.
time += 1
s0Tasks.head.markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber(),
"taskType", TaskResultLost, s0Tasks.head, new ExecutorMetrics, null))
time += 1
val reattempt = createTaskWithNewAttempt(s0Tasks.head, time)
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber(),
reattempt))
// Succeed all tasks in stage 0.
@@ -97,7 +97,7 @@
time += 1
pending.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber(),
"taskType", Success, task, new ExecutorMetrics, TaskMetrics.empty))
}
@@ -122,14 +122,14 @@
val s1Tasks = createTasks(4, execIds, time)
s1Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId,
- stages.last.attemptNumber,
+ stages.last.attemptNumber(),
task))
}
time += 1
s1Tasks.foreach { task =>
task.markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptNumber(),
"taskType", TaskResultLost, task, new ExecutorMetrics, null))
}
@@ -141,7 +141,7 @@
// - Re-submit stage 1, all tasks, and succeed them and the stage.
val oldS1 = stages.last
- val newS1 = new StageInfo(oldS1.stageId, oldS1.attemptNumber + 1, oldS1.name, oldS1.numTasks,
+ val newS1 = new StageInfo(oldS1.stageId, oldS1.attemptNumber() + 1, oldS1.name, oldS1.numTasks,
oldS1.rddInfos, oldS1.parentIds, oldS1.details, oldS1.taskMetrics,
resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)
@@ -152,13 +152,13 @@
val newS1Tasks = createTasks(4, execIds, time)
newS1Tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(newS1.stageId, newS1.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(newS1.stageId, newS1.attemptNumber(), task))
}
time += 1
newS1Tasks.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(newS1.stageId, newS1.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(newS1.stageId, newS1.attemptNumber(), "taskType",
Success, task, new ExecutorMetrics, null))
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index 893f108..b35c576 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -1556,7 +1556,7 @@
provider.checkForLogs()
provider.cleanLogs()
assert(dir.listFiles().size === 1)
- assert(provider.getListing.length === 1)
+ assert(provider.getListing().length === 1)
// Manually delete the appstatus file to make an invalid rolling event log
val appStatusPath = RollingEventLogFilesWriter.getAppStatusFilePath(new Path(writer.logPath),
@@ -1564,7 +1564,7 @@
fs.delete(appStatusPath, false)
provider.checkForLogs()
provider.cleanLogs()
- assert(provider.getListing.length === 0)
+ assert(provider.getListing().length === 0)
// Create a new application
val writer2 = new RollingEventLogFilesWriter("app2", None, dir.toURI, conf, hadoopConf)
@@ -1576,14 +1576,14 @@
// Both folders exist but only one application found
provider.checkForLogs()
provider.cleanLogs()
- assert(provider.getListing.length === 1)
+ assert(provider.getListing().length === 1)
assert(dir.listFiles().size === 2)
// Make sure a new provider sees the valid application
provider.stop()
val newProvider = new FsHistoryProvider(conf)
newProvider.checkForLogs()
- assert(newProvider.getListing.length === 1)
+ assert(newProvider.getListing().length === 1)
}
}
@@ -1613,7 +1613,7 @@
// The 1st checkForLogs should scan/update app2 only since it is newer than app1
provider.checkForLogs()
- assert(provider.getListing.length === 1)
+ assert(provider.getListing().length === 1)
assert(dir.listFiles().size === 2)
assert(provider.getListing().map(e => e.id).contains("app2"))
assert(!provider.getListing().map(e => e.id).contains("app1"))
@@ -1628,7 +1628,7 @@
// The 2nd checkForLogs should scan/update app3 only since it is newer than app1
provider.checkForLogs()
- assert(provider.getListing.length === 2)
+ assert(provider.getListing().length === 2)
assert(dir.listFiles().size === 3)
assert(provider.getListing().map(e => e.id).contains("app3"))
assert(!provider.getListing().map(e => e.id).contains("app1"))
@@ -1655,7 +1655,7 @@
provider.checkForLogs()
provider.cleanLogs()
assert(dir.listFiles().size === 1)
- assert(provider.getListing.length === 1)
+ assert(provider.getListing().length === 1)
// Manually delete event log files and create event log file reader
val eventLogDir = dir.listFiles().head
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index a42fe5f..9aa366d 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -541,7 +541,7 @@
assert(4 === getNumJobsRestful(), s"two jobs back-to-back not updated, server=$server\n")
}
val jobcount = getNumJobs("/jobs")
- assert(!isApplicationCompleted(provider.getListing().next))
+ assert(!isApplicationCompleted(provider.getListing().next()))
listApplications(false) should contain(appId)
@@ -549,7 +549,7 @@
resetSparkContext()
// check the app is now found as completed
eventually(stdTimeout, stdInterval) {
- assert(isApplicationCompleted(provider.getListing().next),
+ assert(isApplicationCompleted(provider.getListing().next()),
s"application never completed, server=$server\n")
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
index afe8ca4..fc6c7d2 100644
--- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
@@ -452,7 +452,7 @@
workerHtml should include ("Spark Worker at")
workerHtml should include ("Running Executors (0)")
verifyStaticResourcesServedByProxy(workerHtml, workerUrl)
- case _ => fail // make sure we don't accidentially skip the tests
+ case _ => fail() // make sure we don't accidentially skip the tests
}
}
diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala
index 909d605..7a70213 100644
--- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala
+++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala
@@ -99,7 +99,7 @@
val rpBuilder = new ResourceProfileBuilder
val ereqs = new ExecutorResourceRequests().resource(GPU, 2)
ereqs.resource(FPGA, 3)
- val rp = rpBuilder.require(ereqs).build
+ val rp = rpBuilder.require(ereqs).build()
testParsingMultipleResources(new SparkConf, rp)
}
@@ -177,7 +177,7 @@
val rpBuilder = new ResourceProfileBuilder
val ereqs = new ExecutorResourceRequests().resource(GPU, 4)
val treqs = new TaskResourceRequests().resource(GPU, 1)
- val rp = rpBuilder.require(ereqs).require(treqs).build
+ val rp = rpBuilder.require(ereqs).require(treqs).build()
testExecutorResourceFoundLessThanRequired(new SparkConf, rp)
}
@@ -245,7 +245,7 @@
val rpBuilder = new ResourceProfileBuilder
val ereqs = new ExecutorResourceRequests().resource(FPGA, 3, scriptPath)
ereqs.resource(GPU, 2)
- val rp = rpBuilder.require(ereqs).build
+ val rp = rpBuilder.require(ereqs).build()
allocatedFileAndConfigsResourceDiscoveryTestFpga(dir, new SparkConf, rp)
}
}
diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorMetricsPollerSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorMetricsPollerSuite.scala
index 11593a0..7a5d32b 100644
--- a/core/src/test/scala/org/apache/spark/executor/ExecutorMetricsPollerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/executor/ExecutorMetricsPollerSuite.scala
@@ -28,7 +28,7 @@
poller.onTaskStart(0L, 0, 0)
// stage (0, 0) has an active task, so it remains on stageTCMP after heartbeat.
- assert(poller.getExecutorUpdates.size === 1)
+ assert(poller.getExecutorUpdates().size === 1)
assert(poller.stageTCMP.size === 1)
assert(poller.stageTCMP.get((0, 0)).count === 1)
@@ -40,9 +40,9 @@
// the next heartbeat will report the peak metrics of stage (0, 0) during the
// previous heartbeat interval, then remove it from stageTCMP.
- assert(poller.getExecutorUpdates.size === 1)
+ assert(poller.getExecutorUpdates().size === 1)
assert(poller.stageTCMP.size === 0)
- poller.stop
+ poller.stop()
}
}
diff --git a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala
index d583afd..a5b4814 100644
--- a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala
@@ -46,8 +46,8 @@
val mockedP = spy[ProcfsMetricsGetter](p)
var ptree: Set[Int] = Set(26109, 22763)
- when(mockedP.computeProcessTree).thenReturn(ptree)
- var r = mockedP.computeAllMetrics
+ when(mockedP.computeProcessTree()).thenReturn(ptree)
+ var r = mockedP.computeAllMetrics()
assert(r.jvmVmemTotal == 4769947648L)
assert(r.jvmRSSTotal == 262610944)
assert(r.pythonVmemTotal == 360595456)
@@ -55,8 +55,8 @@
// proc file of pid 22764 doesn't exist, so partial metrics shouldn't be returned
ptree = Set(26109, 22764, 22763)
- when(mockedP.computeProcessTree).thenReturn(ptree)
- r = mockedP.computeAllMetrics
+ when(mockedP.computeProcessTree()).thenReturn(ptree)
+ r = mockedP.computeAllMetrics()
assert(r.jvmVmemTotal == 0)
assert(r.jvmRSSTotal == 0)
assert(r.pythonVmemTotal == 0)
diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala
index f8217b1..417e711 100644
--- a/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala
+++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala
@@ -68,7 +68,7 @@
createNativeFile(dir, filename, contents, false)
}
// ensure spark job runs successfully without exceptions from the CombineFileInputFormat
- assert(sc.wholeTextFiles(dir.toString).count == 3)
+ assert(sc.wholeTextFiles(dir.toString).count() == 3)
}
}
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
index 5e66ca9..56783de 100644
--- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
@@ -209,7 +209,7 @@
}
private def testAsyncAction[R](action: RDD[Int] => FutureAction[R]): Unit = {
- val executionContextInvoked = Promise[Unit]
+ val executionContextInvoked = Promise[Unit]()
val fakeExecutionContext = new ExecutionContext {
override def execute(runnable: Runnable): Unit = {
executionContextInvoked.success(())
diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
index a204502..2bed5d6 100644
--- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
@@ -81,7 +81,7 @@
1, 100, 3,
(r: ResultSet) => { r.getInt(1) } ).cache()
- assert(rdd.count === 100)
+ assert(rdd.count() === 100)
assert(rdd.reduce(_ + _) === 10100)
}
@@ -93,7 +93,7 @@
"SELECT DATA FROM BIGINT_TEST WHERE ? <= ID AND ID <= ?",
1131544775L, 567279358897692673L, 20,
(r: ResultSet) => { r.getInt(1) } ).cache()
- assert(rdd.count === 100)
+ assert(rdd.count() === 100)
assert(rdd.reduce(_ + _) === 5050)
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala
index da2ccbf..d55acec 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala
@@ -44,7 +44,7 @@
val rdd = sc.makeRDD(Array(1L, 2L, 3L, 4L), 2)
val sampler = new MockSampler
val sample = new PartitionwiseSampledRDD[Long, Long](rdd, sampler, false, 0L)
- assert(sample.distinct().count == 2, "Seeds must be different.")
+ assert(sample.distinct().count() == 2, "Seeds must be different.")
}
test("concurrency") {
diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
index 4e4eafb..2b9dd32 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
@@ -164,7 +164,7 @@
test("pipe with empty partition") {
val data = sc.parallelize(Seq("foo", "bing"), 8)
val piped = data.pipe("wc -c")
- assert(piped.count == 8)
+ assert(piped.count() == 8)
val charCounts = piped.map(_.trim.toInt).collect().toSet
val expected = if (Utils.isWindows) {
// Note that newline character on Windows is \r\n which are two.
diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
index 02ffc23..046017a 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -61,9 +61,9 @@
assert(nums.toLocalIterator.toList === List(1, 2, 3, 4))
val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2)
assert(dups.distinct().count() === 4)
- assert(dups.distinct().count === 4) // Can distinct and count be called without parentheses?
- assert(dups.distinct().collect === dups.distinct().collect)
- assert(dups.distinct(2).collect === dups.distinct().collect)
+ assert(dups.distinct().count() === 4) // Can distinct and count be called without parentheses?
+ assert(dups.distinct().collect() === dups.distinct().collect())
+ assert(dups.distinct(2).collect() === dups.distinct().collect())
assert(nums.reduce(_ + _) === 10)
assert(nums.fold(0)(_ + _) === 10)
assert(nums.map(_.toString).collect().toList === List("1", "2", "3", "4"))
@@ -320,7 +320,7 @@
test("empty RDD") {
val empty = new EmptyRDD[Int](sc)
- assert(empty.count === 0)
+ assert(empty.count() === 0)
assert(empty.collect().size === 0)
val thrown = intercept[UnsupportedOperationException]{
@@ -662,7 +662,7 @@
nums = sc.parallelize(1 to 2, 2)
assert(nums.take(2147483638).size === 2)
- assert(nums.takeAsync(2147483638).get.size === 2)
+ assert(nums.takeAsync(2147483638).get().size === 2)
}
test("top with predefined ordering") {
@@ -1117,7 +1117,7 @@
sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect()
}
// Check that the context has not crashed
- sc.parallelize(1 to 100).map(x => x*2).collect
+ sc.parallelize(1 to 100).map(x => x * 2).collect()
}
/** A contrived RDD that allows the manual addition of dependencies after creation. */
@@ -1165,7 +1165,7 @@
val rdd: RDD[Int] = sc.parallelize(1 to 100)
val rdd2: RDD[Int] = sc.parallelize(1 to 100)
val thrown = intercept[SparkException] {
- rdd.map(x => x * rdd2.count).collect()
+ rdd.map(x => x * rdd2.count()).collect()
}
assert(thrown.getMessage.contains("SPARK-5063"))
}
diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala
index 7149267..ab57f1c 100644
--- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala
@@ -62,7 +62,7 @@
val rprof = new ResourceProfileBuilder()
val gpuExecReq =
new ExecutorResourceRequests().resource("gpu", 2, "someScript")
- val immrprof = rprof.require(gpuExecReq).build
+ val immrprof = rprof.require(gpuExecReq).build()
val error = intercept[SparkException] {
rpmanager.isSupported(immrprof)
}.getMessage()
@@ -82,7 +82,7 @@
val rprof = new ResourceProfileBuilder()
val gpuExecReq =
new ExecutorResourceRequests().resource("gpu", 2, "someScript")
- val immrprof = rprof.require(gpuExecReq).build
+ val immrprof = rprof.require(gpuExecReq).build()
assert(rpmanager.isSupported(immrprof) == true)
}
@@ -97,7 +97,7 @@
val rprof = new ResourceProfileBuilder()
val gpuExecReq =
new ExecutorResourceRequests().resource("gpu", 2, "someScript", "nvidia")
- val immrprof = rprof.require(gpuExecReq).build
+ val immrprof = rprof.require(gpuExecReq).build()
assert(rpmanager.isSupported(immrprof) == true)
}
@@ -178,7 +178,7 @@
val rprof = new ResourceProfileBuilder()
val gpuExecReq =
new ExecutorResourceRequests().resource("gpu", 2, "someScript")
- val immrprof = rprof.require(gpuExecReq).build
+ val immrprof = rprof.require(gpuExecReq).build()
val error = intercept[SparkException] {
rpmanager.isSupported(immrprof)
}.getMessage()
@@ -200,11 +200,11 @@
val treqs = new TaskResourceRequests()
treqs.cpus(i)
rprofBuilder.require(ereqs).require(treqs)
- val rprof = rprofBuilder.build
+ val rprof = rprofBuilder.build()
rpmanager.addResourceProfile(rprof)
if (i == checkId) rpAlreadyExist = Some(rprof)
}
- val rpNotMatch = new ResourceProfileBuilder().build
+ val rpNotMatch = new ResourceProfileBuilder().build()
assert(rpmanager.getEquivalentProfile(rpNotMatch).isEmpty,
s"resourceProfile should not have existed")
@@ -214,7 +214,7 @@
val treqs = new TaskResourceRequests()
treqs.cpus(checkId)
rprofBuilder.require(ereqs).require(treqs)
- val rpShouldMatch = rprofBuilder.build
+ val rpShouldMatch = rprofBuilder.build()
val equivProf = rpmanager.getEquivalentProfile(rpShouldMatch)
assert(equivProf.nonEmpty)
diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala
index 9a2e47e..fd7018f 100644
--- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala
@@ -288,13 +288,13 @@
val taskReq = new TaskResourceRequests().resource("gpu", 1)
val eReq = new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia")
rprofBuilder.require(taskReq).require(eReq)
- val rprof = rprofBuilder.build
+ val rprof = rprofBuilder.build()
val rprofBuilder2 = new ResourceProfileBuilder()
val taskReq2 = new TaskResourceRequests().resource("gpu", 1)
val eReq2 = new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia")
rprofBuilder2.require(taskReq2).require(eReq2)
- val rprof2 = rprofBuilder.build
+ val rprof2 = rprofBuilder.build()
rprof2.setResourceProfileId(rprof.id)
assert(rprof === rprof2, "resource profile equality not working")
diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala
index ffe5ff5..1ab9f7c 100644
--- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala
@@ -166,7 +166,7 @@
val ereqs = new ExecutorResourceRequests().resource(GPU, 2, gpuDiscovery)
val treqs = new TaskResourceRequests().resource(GPU, 1)
- val rp = rpBuilder.require(ereqs).require(treqs).build
+ val rp = rpBuilder.require(ereqs).require(treqs).build()
val resourcesFromBoth = getOrDiscoverAllResourcesForResourceProfile(
Some(resourcesFile), SPARK_EXECUTOR_PREFIX, rp, conf)
val expectedGpuInfo = new ResourceInformation(GPU, Array("0", "1"))
diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala
index 26cd537..849832c 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala
@@ -81,7 +81,7 @@
val rdd = sc.makeRDD(1 to 10, 4)
val rdd2 = rdd.barrier().mapPartitions { it =>
val context = BarrierTaskContext.get()
- val partitionId = context.partitionId
+ val partitionId = context.partitionId()
if (partitionId == 0) {
context.barrier()
} else {
@@ -155,7 +155,7 @@
val rdd2 = rdd.barrier().mapPartitions { it =>
val context = BarrierTaskContext.get()
// Task 3 shall sleep 2000ms to ensure barrier() call timeout
- if (context.taskAttemptId == 3) {
+ if (context.taskAttemptId() == 3) {
Thread.sleep(2000)
}
context.barrier()
@@ -175,7 +175,7 @@
val rdd = sc.makeRDD(1 to 10, 4)
val rdd2 = rdd.barrier().mapPartitions { it =>
val context = BarrierTaskContext.get()
- if (context.taskAttemptId != 0) {
+ if (context.taskAttemptId() != 0) {
context.barrier()
}
it
@@ -195,7 +195,7 @@
val rdd2 = rdd.barrier().mapPartitions { it =>
val context = BarrierTaskContext.get()
try {
- if (context.taskAttemptId == 0) {
+ if (context.taskAttemptId() == 0) {
// Due to some non-obvious reason, the code can trigger an Exception and skip the
// following statements within the try ... catch block, including the first barrier()
// call.
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 7691b98..5040532 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -457,7 +457,7 @@
* directly through CompletionEvents.
*/
private val jobComputeFunc = (context: TaskContext, it: Iterator[(_)]) =>
- it.next.asInstanceOf[Tuple2[_, _]]._1
+ it.next().asInstanceOf[Tuple2[_, _]]._1
/** Send the given CompletionEvent messages for the tasks in the TaskSet. */
private def complete(taskSet: TaskSet, taskEndInfos: Seq[(TaskEndReason, Any)]): Unit = {
@@ -3449,12 +3449,12 @@
test("test 1 resource profile") {
val ereqs = new ExecutorResourceRequests().cores(4)
val treqs = new TaskResourceRequests().cpus(1)
- val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build
+ val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build()
val rdd = sc.parallelize(1 to 10).map(x => (x, x)).withResources(rp1)
val (shuffledeps, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd)
val rpMerged = scheduler.mergeResourceProfilesForStage(resourceprofiles)
- val expectedid = Option(rdd.getResourceProfile).map(_.id)
+ val expectedid = Option(rdd.getResourceProfile()).map(_.id)
assert(expectedid.isDefined)
assert(expectedid.get != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)
assert(rpMerged.id == expectedid.get)
@@ -3464,11 +3464,11 @@
import org.apache.spark.resource._
val ereqs = new ExecutorResourceRequests().cores(4)
val treqs = new TaskResourceRequests().cpus(1)
- val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build
+ val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build()
val ereqs2 = new ExecutorResourceRequests().cores(2)
val treqs2 = new TaskResourceRequests().cpus(2)
- val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build
+ val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build()
val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2)
val error = intercept[IllegalArgumentException] {
@@ -3484,11 +3484,11 @@
val ereqs = new ExecutorResourceRequests().cores(4)
val treqs = new TaskResourceRequests().cpus(1)
- val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build
+ val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build()
val ereqs2 = new ExecutorResourceRequests().cores(2)
val treqs2 = new TaskResourceRequests().cpus(2)
- val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build
+ val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build()
val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2)
val (shuffledeps, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd)
@@ -3502,11 +3502,11 @@
val ereqs = new ExecutorResourceRequests().cores(4)
val treqs = new TaskResourceRequests().cpus(1)
- val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build
+ val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build()
val ereqs2 = new ExecutorResourceRequests().cores(2)
val treqs2 = new TaskResourceRequests().cpus(2)
- val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build
+ val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build()
val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2)
val (_, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd)
@@ -3642,10 +3642,10 @@
import org.apache.spark.resource._
val ereqs = new ExecutorResourceRequests().cores(4)
val treqs = new TaskResourceRequests().cpus(1)
- val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build
+ val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build()
val ereqs2 = new ExecutorResourceRequests().cores(6)
val treqs2 = new TaskResourceRequests().cpus(2)
- val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build
+ val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build()
val rddWithRp = new MyRDD(sc, 2, Nil).withResources(rp1)
val rddA = new MyRDD(sc, 2, Nil).withResources(rp1)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index bd65936..cd8fac2 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -173,7 +173,7 @@
.mapPartitions(_.map(elem => (elem, elem)))
.filter(elem => elem._1 % 2 == 0)
.reduceByKey(_ + _)
- .collect
+ .collect()
sc.stop()
val eventLogStream = EventLogFileReader.openEventLog(new Path(testDirPath, appId), fileSystem)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala
index 45da750..fcacd22 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala
@@ -59,7 +59,7 @@
private class ThrowExceptionOnFirstAttemptOutputCommitter extends FileOutputCommitter {
override def commitTask(context: TaskAttemptContext): Unit = {
val ctx = TaskContext.get()
- if (ctx.attemptNumber < 1) {
+ if (ctx.attemptNumber() < 1) {
throw new java.io.FileNotFoundException("Intentional exception")
}
super.commitTask(context)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
index 842a261..0533f9d 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
@@ -304,7 +304,7 @@
def failFirstCommitAttempt(iter: Iterator[Int]): Unit = {
val ctx = TaskContext.get()
runCommitWithProvidedCommitter(ctx, iter,
- if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter)
+ if (ctx.attemptNumber() == 0) failingOutputCommitter else successfulOutputCommitter)
}
private def runCommitWithProvidedCommitter(
@@ -324,9 +324,9 @@
// Create TaskAttemptContext.
// Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
// around by taking a mod. We expect that no task will be attempted 2 billion times.
- val taskAttemptId = (ctx.taskAttemptId % Int.MaxValue).toInt
+ val taskAttemptId = (ctx.taskAttemptId() % Int.MaxValue).toInt
val attemptId = new TaskAttemptID(
- new TaskID(jobId.value, TaskType.MAP, ctx.partitionId), taskAttemptId)
+ new TaskID(jobId.value, TaskType.MAP, ctx.partitionId()), taskAttemptId)
val taskContext = new TaskAttemptContextImpl(jobConf, attemptId)
committer.setupTask(taskContext)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
index 54a42c1..ff5d821 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
@@ -340,13 +340,13 @@
sc = new SparkContext("local[1,2]", "test") // use maxRetries = 2 because we test failed tasks
// Check that attemptIds are 0 for all tasks' initial attempts
val attemptIds = sc.parallelize(Seq(1, 2), 2).mapPartitions { iter =>
- Seq(TaskContext.get().attemptNumber).iterator
+ Seq(TaskContext.get().attemptNumber()).iterator
}.collect()
assert(attemptIds.toSet === Set(0))
// Test a job with failed tasks
val attemptIdsWithFailedTask = sc.parallelize(Seq(1, 2), 2).mapPartitions { iter =>
- val attemptId = TaskContext.get().attemptNumber
+ val attemptId = TaskContext.get().attemptNumber()
if (iter.next() == 1 && attemptId == 0) {
throw new Exception("First execution of task failed")
}
@@ -385,7 +385,7 @@
for (numPartitions <- 1 to 10) {
val numPartitionsFromContext = sc.parallelize(1 to 1000, numPartitions)
.mapPartitions { _ =>
- Seq(TaskContext.get.numPartitions()).iterator
+ Seq(TaskContext.get().numPartitions()).iterator
}.collect()
assert(numPartitionsFromContext.toSet === Set(numPartitions),
s"numPartitions = $numPartitions")
@@ -394,7 +394,7 @@
for (numPartitions <- 1 to 10) {
val numPartitionsFromContext = sc.parallelize(1 to 1000, 2).repartition(numPartitions)
.mapPartitions { _ =>
- Seq(TaskContext.get.numPartitions()).iterator
+ Seq(TaskContext.get().numPartitions()).iterator
}.collect()
assert(numPartitionsFromContext.toSet === Set(numPartitions),
s"numPartitions = $numPartitions")
@@ -411,7 +411,7 @@
sc.parallelize(1 to 10, 10).map { i =>
acc1.add(1)
acc2.add(1)
- if (TaskContext.get.attemptNumber() <= 2) {
+ if (TaskContext.get().attemptNumber() <= 2) {
throw new Exception("you did something wrong")
} else {
0
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
index 3ea0843..5a8722a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
@@ -99,7 +99,7 @@
extends TaskResultGetter(env, scheduler) {
// Use the current thread so we can access its results synchronously
- protected override val getTaskResultExecutor = ThreadUtils.sameThreadExecutorService
+ protected override val getTaskResultExecutor = ThreadUtils.sameThreadExecutorService()
// DirectTaskResults that we receive from the executors
private val _taskResults = new ArrayBuffer[DirectTaskResult[_]]
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoIteratorBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoIteratorBenchmark.scala
index 5de1a12..f526428 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoIteratorBenchmark.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoIteratorBenchmark.scala
@@ -99,15 +99,15 @@
}
}
- createCase("int", 1, Random.nextInt)
- createCase("int", 10, Random.nextInt)
- createCase("int", 100, Random.nextInt)
+ createCase("int", 1, Random.nextInt())
+ createCase("int", 10, Random.nextInt())
+ createCase("int", 100, Random.nextInt())
createCase("string", 1, Random.nextString(5))
createCase("string", 10, Random.nextString(5))
createCase("string", 100, Random.nextString(5))
- createCase("Array[int]", 1, Array.fill(10)(Random.nextInt))
- createCase("Array[int]", 10, Array.fill(10)(Random.nextInt))
- createCase("Array[int]", 100, Array.fill(10)(Random.nextInt))
+ createCase("Array[int]", 1, Array.fill(10)(Random.nextInt()))
+ createCase("Array[int]", 10, Array.fill(10)(Random.nextInt()))
+ createCase("Array[int]", 100, Array.fill(10)(Random.nextInt()))
}
def createSerializer(): SerializerInstance = {
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index 40319e1..86dc6c8 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -290,11 +290,11 @@
}
test("kryo with parallelize for specialized tuples") {
- assert(sc.parallelize(Seq((1, 11), (2, 22), (3, 33))).count === 3)
+ assert(sc.parallelize(Seq((1, 11), (2, 22), (3, 33))).count() === 3)
}
test("kryo with parallelize for primitive arrays") {
- assert(sc.parallelize(Array(1, 2, 3)).count === 3)
+ assert(sc.parallelize(Array(1, 2, 3)).count() === 3)
}
test("kryo with collect for specialized tuples") {
@@ -425,11 +425,11 @@
test("getAutoReset") {
val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance]
- assert(ser.getAutoReset)
+ assert(ser.getAutoReset())
val conf = new SparkConf().set(KRYO_USER_REGISTRATORS,
Seq(classOf[RegistratorWithoutAutoReset].getName))
val ser2 = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance]
- assert(!ser2.getAutoReset)
+ assert(!ser2.getAutoReset())
}
test("SPARK-25176 ClassCastException when writing a Map after previously " +
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
index f1e987a..4d75f5d 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -206,7 +206,7 @@
val s1Tasks = createTasks(4, execIds)
s1Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId,
- stages.head.attemptNumber,
+ stages.head.attemptNumber(),
task))
}
@@ -225,7 +225,7 @@
check[TaskDataWrapper](task.taskId) { wrapper =>
assert(wrapper.taskId === task.taskId)
assert(wrapper.stageId === stages.head.stageId)
- assert(wrapper.stageAttemptId === stages.head.attemptNumber)
+ assert(wrapper.stageAttemptId === stages.head.attemptNumber())
assert(wrapper.index === task.index)
assert(wrapper.attempt === task.attemptNumber)
assert(wrapper.launchTime === task.launchTime)
@@ -246,7 +246,7 @@
Some(value), None, true, false, None)
listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(
task.executorId,
- Seq((task.taskId, stages.head.stageId, stages.head.attemptNumber, Seq(accum)))))
+ Seq((task.taskId, stages.head.stageId, stages.head.attemptNumber(), Seq(accum)))))
}
check[StageDataWrapper](key(stages.head)) { stage =>
@@ -268,7 +268,7 @@
executorId = execIds.head,
taskFailures = 2,
stageId = stages.head.stageId,
- stageAttemptId = stages.head.attemptNumber))
+ stageAttemptId = stages.head.attemptNumber()))
val executorStageSummaryWrappers =
KVUtils.viewToSeq(store.view(classOf[ExecutorStageSummaryWrapper]).index("stage")
@@ -296,7 +296,7 @@
hostId = "2.example.com", // this is where the second executor is hosted
executorFailures = 1,
stageId = stages.head.stageId,
- stageAttemptId = stages.head.attemptNumber))
+ stageAttemptId = stages.head.attemptNumber()))
val executorStageSummaryWrappersForNode =
KVUtils.viewToSeq(store.view(classOf[ExecutorStageSummaryWrapper]).index("stage")
@@ -314,12 +314,12 @@
// Fail one of the tasks, re-start it.
time += 1
s1Tasks.head.markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber(),
"taskType", TaskResultLost, s1Tasks.head, new ExecutorMetrics, null))
time += 1
val reattempt = newAttempt(s1Tasks.head, nextTaskId())
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber(),
reattempt))
assert(store.count(classOf[TaskDataWrapper]) === s1Tasks.size + 1)
@@ -354,7 +354,7 @@
val killed = s1Tasks.drop(1).head
killed.finishTime = time
killed.failed = true
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber(),
"taskType", TaskKilled("killed"), killed, new ExecutorMetrics, null))
check[JobDataWrapper](1) { job =>
@@ -376,13 +376,13 @@
time += 1
val denied = newAttempt(killed, nextTaskId())
val denyReason = TaskCommitDenied(1, 1, 1)
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber(),
denied))
time += 1
denied.finishTime = time
denied.failed = true
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber(),
"taskType", denyReason, denied, new ExecutorMetrics, null))
check[JobDataWrapper](1) { job =>
@@ -402,7 +402,7 @@
// Start a new attempt.
val reattempt2 = newAttempt(denied, nextTaskId())
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber(),
reattempt2))
// Succeed all tasks in stage 1.
@@ -415,7 +415,7 @@
time += 1
pending.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber(),
"taskType", Success, task, new ExecutorMetrics, s1Metrics))
}
@@ -492,7 +492,7 @@
hostId = "1.example.com",
executorFailures = 1,
stageId = stages.last.stageId,
- stageAttemptId = stages.last.attemptNumber))
+ stageAttemptId = stages.last.attemptNumber()))
check[ExecutorSummaryWrapper](execIds.head) { exec =>
assert(exec.info.blacklistedInStages === Set(stages.last.stageId))
@@ -504,14 +504,14 @@
val s2Tasks = createTasks(4, execIds)
s2Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId,
- stages.last.attemptNumber,
+ stages.last.attemptNumber(),
task))
}
time += 1
s2Tasks.foreach { task =>
task.markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptNumber(),
"taskType", TaskResultLost, task, new ExecutorMetrics, null))
}
@@ -546,7 +546,7 @@
// - Re-submit stage 2, all tasks, and succeed them and the stage.
val oldS2 = stages.last
- val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptNumber + 1, oldS2.name, oldS2.numTasks,
+ val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptNumber() + 1, oldS2.name, oldS2.numTasks,
oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics,
resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)
@@ -558,13 +558,13 @@
val newS2Tasks = createTasks(4, execIds)
newS2Tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptNumber(), task))
}
time += 1
newS2Tasks.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptNumber(), "taskType",
Success, task, new ExecutorMetrics, null))
}
@@ -617,14 +617,14 @@
j2s2Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(j2Stages.last.stageId,
- j2Stages.last.attemptNumber,
+ j2Stages.last.attemptNumber(),
task))
}
time += 1
j2s2Tasks.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(j2Stages.last.stageId, j2Stages.last.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(j2Stages.last.stageId, j2Stages.last.attemptNumber(),
"taskType", Success, task, new ExecutorMetrics, null))
}
@@ -1077,17 +1077,17 @@
// task end event.
time += 1
val task = createTasks(1, Array("1")).head
- listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptNumber(), task))
time += 1
task.markFinished(TaskState.FINISHED, time)
val metrics = TaskMetrics.empty
metrics.setExecutorRunTime(42L)
- listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptNumber(),
"taskType", Success, task, new ExecutorMetrics, metrics))
new AppStatusStore(store)
- .taskSummary(dropped.stageId, dropped.attemptNumber, Array(0.25d, 0.50d, 0.75d))
+ .taskSummary(dropped.stageId, dropped.attemptNumber(), Array(0.25d, 0.50d, 0.75d))
assert(store.count(classOf[CachedQuantile], "stage", key(dropped)) === 3)
stages.drop(1).foreach { s =>
@@ -1123,13 +1123,13 @@
time += 1
val tasks = createTasks(2, Array("1"))
tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber(), task))
}
assert(store.count(classOf[TaskDataWrapper]) === 2)
// Start a 3rd task. The finished tasks should be deleted.
createTasks(1, Array("1")).foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber(), task))
}
assert(store.count(classOf[TaskDataWrapper]) === 2)
intercept[NoSuchElementException] {
@@ -1138,7 +1138,7 @@
// Start a 4th task. The first task should be deleted, even if it's still running.
createTasks(1, Array("1")).foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber(), task))
}
assert(store.count(classOf[TaskDataWrapper]) === 2)
intercept[NoSuchElementException] {
@@ -1258,23 +1258,23 @@
// Start task 1 and task 2
val tasks = createTasks(3, Array("1"))
tasks.take(2).foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber(), task))
}
// Stop task 2 before task 1
time += 1
tasks(1).markFinished(TaskState.FINISHED, time)
listener.onTaskEnd(SparkListenerTaskEnd(
- stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(1),
+ stage1.stageId, stage1.attemptNumber(), "taskType", Success, tasks(1),
new ExecutorMetrics, null))
time += 1
tasks(0).markFinished(TaskState.FINISHED, time)
listener.onTaskEnd(SparkListenerTaskEnd(
- stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0),
+ stage1.stageId, stage1.attemptNumber(), "taskType", Success, tasks(0),
new ExecutorMetrics, null))
// Start task 3 and task 2 should be evicted.
- listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, tasks(2)))
+ listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber(), tasks(2)))
assert(store.count(classOf[TaskDataWrapper]) === 2)
intercept[NoSuchElementException] {
store.read(classOf[TaskDataWrapper], tasks(1).id)
@@ -1335,14 +1335,14 @@
// Start 2 Tasks
val tasks = createTasks(2, Array("1"))
tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber(), task))
}
// Task 1 Finished
time += 1
tasks(0).markFinished(TaskState.FINISHED, time)
listener.onTaskEnd(SparkListenerTaskEnd(
- stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0),
+ stage1.stageId, stage1.attemptNumber(), "taskType", Success, tasks(0),
new ExecutorMetrics, null))
// Stage 1 Completed
@@ -1357,7 +1357,7 @@
time += 1
tasks(1).markFinished(TaskState.FINISHED, time)
listener.onTaskEnd(
- SparkListenerTaskEnd(stage1.stageId, stage1.attemptNumber, "taskType",
+ SparkListenerTaskEnd(stage1.stageId, stage1.attemptNumber(), "taskType",
TaskKilled(reason = "Killed"), tasks(1), new ExecutorMetrics, null))
// Ensure killed task metrics are updated
@@ -1395,16 +1395,16 @@
val tasks = createTasks(4, Array("1", "2"))
tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber(), task))
}
time += 1
tasks(0).markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
Success, tasks(0), new ExecutorMetrics, null))
time += 1
tasks(1).markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
Success, tasks(1), new ExecutorMetrics, null))
stage.failureReason = Some("Failed")
@@ -1415,12 +1415,12 @@
time += 1
tasks(2).markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), new ExecutorMetrics,
null))
time += 1
tasks(3).markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), new ExecutorMetrics,
null))
@@ -1865,38 +1865,38 @@
val tasks = createTasks(2, Array("1", "2"))
tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber(), task))
}
time += 1
tasks(0).markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
Success, tasks(0), new ExecutorMetrics, null))
// executor lost, success task will be resubmitted
time += 1
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
Resubmitted, tasks(0), new ExecutorMetrics, null))
// executor lost, running task will be failed and rerun
time += 1
tasks(1).markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
ExecutorLostFailure("1", true, Some("Lost executor")), tasks(1), new ExecutorMetrics,
null))
tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task))
+ listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber(), task))
}
time += 1
tasks(0).markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
Success, tasks(0), new ExecutorMetrics, null))
time += 1
tasks(1).markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(), "taskType",
Success, tasks(1), new ExecutorMetrics, null))
listener.onStageCompleted(SparkListenerStageCompleted(stage))
@@ -1930,7 +1930,7 @@
assert(job.numActiveStages == 0)
}
- private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptNumber)
+ private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptNumber())
private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = {
val value = store.read(classTag[T].runtimeClass, key).asInstanceOf[T]
diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala
index f93c2bc..e7d78cb 100644
--- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala
+++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala
@@ -177,7 +177,7 @@
s1Tasks.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber,
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber(),
"taskType", Success, task, new ExecutorMetrics, s1Metrics))
}
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index 2af0f9c..a5e9a58 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -2212,7 +2212,7 @@
val sortedBlocks = blocks.sortBy(b => (b.shuffleId, b.mapId))
val resolver = mock(classOf[MigratableResolver])
- when(resolver.getStoredShuffles).thenReturn(blocks)
+ when(resolver.getStoredShuffles()).thenReturn(blocks)
val bm = mock(classOf[BlockManager])
when(bm.migratableResolver).thenReturn(resolver)
diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
index 48610cb..eacd1ee 100644
--- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
@@ -83,7 +83,7 @@
val ids = (1 to 100).map(i => TestBlockId("test_" + i))
val files = ids.map(id => diskBlockManager.getFile(id))
files.foreach(file => writeToFile(file, 10))
- assert(diskBlockManager.getAllBlocks.toSet === ids.toSet)
+ assert(diskBlockManager.getAllBlocks().toSet === ids.toSet)
}
test("SPARK-22227: non-block files are skipped") {
diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala
index fd99a03..d4f04e8 100644
--- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala
@@ -609,7 +609,7 @@
assert(putIteratorAsValues("b1", nativeObjIterator, ClassTag.Any).isRight)
assert(putIteratorAsValues("b2", nativeObjIterator, ClassTag.Any).isRight)
- memoryStore.clear
+ memoryStore.clear()
// Check if allocator was cleared.
while (allocator.getAllocatedMemory > 0) {
Thread.sleep(500)
diff --git a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
index 185d91e..b0f1fca 100644
--- a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
@@ -57,7 +57,7 @@
withSpark(newSparkContext()) { sc =>
sc.setLocalProperty(CallSite.LONG_FORM, "collect at <console>:25")
sc.setLocalProperty(CallSite.SHORT_FORM, "collect at <console>:25")
- sc.parallelize(1 to 10).collect
+ sc.parallelize(1 to 10).collect()
eventually(timeout(10.seconds), interval(50.milliseconds)) {
goToUi(sc, "/jobs")
@@ -81,7 +81,7 @@
test("SPARK-31882: Link URL for Stage DAGs should not depend on paged table.") {
withSpark(newSparkContext()) { sc =>
- sc.parallelize(1 to 100).map(v => (v, v)).repartition(10).reduceByKey(_ + _).collect
+ sc.parallelize(1 to 100).map(v => (v, v)).repartition(10).reduceByKey(_ + _).collect()
eventually(timeout(10.seconds), interval(50.microseconds)) {
val pathWithPagedTable =
@@ -103,7 +103,7 @@
test("SPARK-31886: Color barrier execution mode RDD correctly") {
withSpark(newSparkContext()) { sc =>
- sc.parallelize(1 to 10).barrier.mapPartitions(identity).repartition(1).collect()
+ sc.parallelize(1 to 10).barrier().mapPartitions(identity).repartition(1).collect()
eventually(timeout(10.seconds), interval(50.milliseconds)) {
goToUi(sc, "/jobs/job/?id=0")
@@ -131,7 +131,7 @@
test("Search text for paged tables should not be saved") {
withSpark(newSparkContext()) { sc =>
- sc.parallelize(1 to 10).collect
+ sc.parallelize(1 to 10).collect()
eventually(timeout(10.seconds), interval(1.seconds)) {
val taskSearchBox = "$(\"input[aria-controls='active-tasks-table']\")"
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index 7e74cc9..a2d1293 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -340,7 +340,7 @@
data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle
// Simulate fetch failures:
val mappedData = data.map { x =>
- val taskContext = TaskContext.get
+ val taskContext = TaskContext.get()
if (taskContext.taskAttemptId() == 1) {
// Cause the post-shuffle stage to fail on its first attempt with a single task failure
val env = SparkEnv.get
@@ -820,10 +820,10 @@
test("description for empty jobs") {
withSpark(newSparkContext()) { sc =>
- sc.emptyRDD[Int].collect
+ sc.emptyRDD[Int].collect()
val description = "This is my job"
sc.setJobDescription(description)
- sc.emptyRDD[Int].collect
+ sc.emptyRDD[Int].collect()
eventually(timeout(10.seconds), interval(50.milliseconds)) {
goToUi(sc, "/jobs")
diff --git a/core/src/test/scala/org/apache/spark/ui/env/EnvironmentPageSuite.scala b/core/src/test/scala/org/apache/spark/ui/env/EnvironmentPageSuite.scala
index 9279187..a318e08 100644
--- a/core/src/test/scala/org/apache/spark/ui/env/EnvironmentPageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/env/EnvironmentPageSuite.scala
@@ -43,8 +43,8 @@
when(info.classpathEntries).thenReturn(Seq.empty)
val store = mock(classOf[AppStatusStore])
- when(store.environmentInfo).thenReturn(info)
- when(store.resourceProfileInfo).thenReturn(Seq.empty)
+ when(store.environmentInfo()).thenReturn(info)
+ when(store.resourceProfileInfo()).thenReturn(Seq.empty)
val environmentPage = new EnvironmentPage(environmentTab, new SparkConf, store)
val request = mock(classOf[HttpServletRequest])
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index 5a6c332..3defd4b 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -164,7 +164,7 @@
.resource("gpu", 2, "myscript")
.resource("myCustomResource", amount = Int.MaxValue + 1L, discoveryScript = "myscript2")
rprofBuilder.require(taskReq).require(execReq)
- val resourceProfile = rprofBuilder.build
+ val resourceProfile = rprofBuilder.build()
resourceProfile.setResourceProfileId(21)
val resourceProfileAdded = SparkListenerResourceProfileAdded(resourceProfile)
testEvent(stageSubmitted, stageSubmittedJsonString)
diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala
index 4909d5f..efa0da4 100644
--- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala
@@ -30,7 +30,7 @@
test("one iteration") {
val i = new StubIterator(Buffer(1))
i.hasNext should be (true)
- i.next should be (1)
+ i.next() should be (1)
i.hasNext should be (false)
intercept[NoSuchElementException] { i.next() }
}
@@ -38,9 +38,9 @@
test("two iterations") {
val i = new StubIterator(Buffer(1, 2))
i.hasNext should be (true)
- i.next should be (1)
+ i.next() should be (1)
i.hasNext should be (true)
- i.next should be (2)
+ i.next() should be (2)
i.hasNext should be (false)
intercept[NoSuchElementException] { i.next() }
}
@@ -60,8 +60,8 @@
test("close is called once for non-empty iterations") {
val i = new StubIterator(Buffer(1, 2))
- i.next should be (1)
- i.next should be (2)
+ i.next() should be (1)
+ i.next() should be (2)
// close isn't called until we check for the next element
i.closeCalled should be (0)
i.hasNext should be (false)
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 58ce15c..25f48c7 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -1253,7 +1253,7 @@
assert(isErrorOccurred)
// if the try, catch and finally blocks don't throw exceptions
Utils.tryWithSafeFinallyAndFailureCallbacks {}(catchBlock = {}, finallyBlock = {})
- TaskContext.unset
+ TaskContext.unset()
}
test("load extensions") {
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 4b63f1d..fb15028 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
@@ -451,7 +451,7 @@
}
val first50Keys = for ( _ <- 0 until 50) yield {
- val (k, vs) = it.next
+ val (k, vs) = it.next()
val sortedVs = vs.sorted
assert(sortedVs.seq == (0 until 10).map(10 * k + _))
k
@@ -472,7 +472,7 @@
val next50Keys = for ( _ <- 0 until 50) yield {
- val (k, vs) = it.next
+ val (k, vs) = it.next()
val sortedVs = vs.sorted
assert(sortedVs.seq == (0 until 10).map(10 * k + _))
k
diff --git a/core/src/test/scala/org/apache/spark/util/collection/PercentileHeapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PercentileHeapSuite.scala
index fd1208c..20def45 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/PercentileHeapSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/PercentileHeapSuite.scala
@@ -26,7 +26,7 @@
test("When PercentileHeap is empty, NoSuchElementException is thrown.") {
val medianHeap = new PercentileHeap(0.5)
intercept[NoSuchElementException] {
- medianHeap.percentile
+ medianHeap.percentile()
}
}
@@ -38,8 +38,8 @@
private def testPercentileFor(nums: Seq[Int], percentage: Double) = {
val h = new PercentileHeap(percentage)
Random.shuffle(nums).foreach(h.insert(_))
- assert(h.size == nums.length)
- assert(h.percentile == percentile(nums, percentage))
+ assert(h.size() == nums.length)
+ assert(h.percentile() == percentile(nums, percentage))
}
private val tests = Seq(
@@ -68,7 +68,7 @@
val h = new PercentileHeap(0.95)
shuffled.foreach { x =>
h.insert(x)
- for (_ <- 0 until h.size) h.percentile
+ for (_ <- 0 until h.size()) h.percentile()
}
System.nanoTime() - start
}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala
index d33ac97..f5d417f 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala
@@ -136,7 +136,7 @@
def randomBitMask(rand: Random): Long = {
var tmp = ~0L
for (i <- 0 to rand.nextInt(5)) {
- tmp &= rand.nextLong
+ tmp &= rand.nextLong()
}
tmp
}
diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala
index eb1aab6..8fcbfd9 100644
--- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala
@@ -731,7 +731,7 @@
val s1 = sampler.sample(data.iterator).toArray
s1.length should be > 0
- sampler = base.cloneComplement
+ sampler = base.cloneComplement()
sampler.setSeed(seed)
val s2 = sampler.sample(data.iterator).toArray
s2.length should be > 0
diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
index 323bab4..5dfb8ec 100644
--- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
@@ -103,7 +103,7 @@
println("Creating SparkSession")
val spark = SparkSession
- .builder
+ .builder()
.appName("DFS Read Write Test")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
index 6e95318..ae55048 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
@@ -22,12 +22,12 @@
object ExceptionHandlingTest {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("ExceptionHandlingTest")
.getOrCreate()
spark.sparkContext.parallelize(0 until spark.sparkContext.defaultParallelism).foreach { i =>
- if (math.random > 0.75) {
+ if (math.random() > 0.75) {
throw new Exception("Testing exception handling")
}
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
index c07c1af..8368fca 100644
--- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
@@ -28,7 +28,7 @@
object GroupByTest {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("GroupBy Test")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
index 4869867..6443bbe 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
@@ -32,7 +32,7 @@
System.exit(1)
}
val spark = SparkSession
- .builder
+ .builder()
.appName("HdfsTest")
.getOrCreate()
val file = spark.read.text(args(0)).rdd
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
index 87c2f68..06c9173 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
@@ -134,10 +134,10 @@
}
private def randomVector(n: Int): RealVector =
- new ArrayRealVector(Array.fill(n)(math.random))
+ new ArrayRealVector(Array.fill(n)(math.random()))
private def randomMatrix(rows: Int, cols: Int): RealMatrix =
- new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random))
+ new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random()))
}
// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
index 7660ffd..c77d99c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
@@ -24,8 +24,8 @@
def main(args: Array[String]): Unit = {
var count = 0
for (i <- 1 to 100000) {
- val x = random * 2 - 1
- val y = random * 2 - 1
+ val x = random() * 2 - 1
+ val y = random() * 2 - 1
if (x*x + y*y <= 1) count += 1
}
println(s"Pi is roughly ${4 * count / 100000.0}")
diff --git a/examples/src/main/scala/org/apache/spark/examples/MiniReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/MiniReadWriteTest.scala
index aa88cd5..c003dc8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MiniReadWriteTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MiniReadWriteTest.scala
@@ -96,7 +96,7 @@
println("Creating SparkSession")
val spark = SparkSession
- .builder
+ .builder()
.appName("Mini Read Write Test")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index 0692c51..d6427cd 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -29,7 +29,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("Multi-Broadcast Test")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
index 2bd7c3e..086a004 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
@@ -28,7 +28,7 @@
object SimpleSkewedGroupByTest {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("SimpleSkewedGroupByTest")
.getOrCreate()
@@ -50,16 +50,16 @@
result(i) = (offset, byteArr)
} else {
// generate a key for one of the other reducers
- val key = 1 + ranGen.nextInt(numReducers-1) + offset
+ val key = 1 + ranGen.nextInt(numReducers - 1) + offset
result(i) = (key, byteArr)
}
}
result
- }.cache
+ }.cache()
// Enforce that everything has been calculated and in cache
- pairs1.count
+ pairs1.count()
- println(s"RESULT: ${pairs1.groupByKey(numReducers).count}")
+ println(s"RESULT: ${pairs1.groupByKey(numReducers).count()}")
spark.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
index 2e7abd6..9001ab0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
@@ -28,7 +28,7 @@
object SkewedGroupByTest {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("GroupBy Test")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
index 651f022..42cc0a8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
@@ -109,7 +109,7 @@
println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS")
val spark = SparkSession
- .builder
+ .builder()
.appName("SparkALS")
.getOrCreate()
@@ -141,10 +141,10 @@
}
private def randomVector(n: Int): RealVector =
- new ArrayRealVector(Array.fill(n)(math.random))
+ new ArrayRealVector(Array.fill(n)(math.random()))
private def randomMatrix(rows: Int, cols: Int): RealMatrix =
- new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random))
+ new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random()))
}
// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
index 8c09ce6..7bbe594 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
@@ -67,7 +67,7 @@
showWarning()
val spark = SparkSession
- .builder
+ .builder()
.appName("SparkHdfsLR")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
index e0ab07a..a6e1de7 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -67,7 +67,7 @@
showWarning()
val spark = SparkSession
- .builder
+ .builder()
.appName("SparkKMeans")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
index deb6668..469e429 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
@@ -64,7 +64,7 @@
showWarning()
val spark = SparkSession
- .builder
+ .builder()
.appName("SparkLR")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
index 3bd475c..27e1d46 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
@@ -56,7 +56,7 @@
showWarning()
val spark = SparkSession
- .builder
+ .builder()
.appName("SparkPageRank")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
index a8eec6a..468bc91 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
@@ -26,14 +26,14 @@
object SparkPi {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("Spark Pi")
.getOrCreate()
val slices = if (args.length > 0) args(0).toInt else 2
val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow
val count = spark.sparkContext.parallelize(1 until n, slices).map { i =>
- val x = random * 2 - 1
- val y = random * 2 - 1
+ val x = random() * 2 - 1
+ val y = random() * 2 - 1
if (x*x + y*y <= 1) 1 else 0
}.reduce(_ + _)
println(s"Pi is roughly ${4.0 * count / (n - 1)}")
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
index 7a6fa9a..11f730c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
@@ -43,7 +43,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("SparkTC")
.getOrCreate()
val slices = if (args.length > 0) args(0).toInt else 2
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala
index 8441b5a..33f7156 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala
@@ -37,7 +37,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession.
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
val sc = spark.sparkContext
@@ -63,7 +63,7 @@
olderFollowers.mapValues( (id, value) =>
value match { case (count, totalAge) => totalAge / count } )
// Display the results
- avgAgeOfOlderFollowers.collect.foreach(println(_))
+ avgAgeOfOlderFollowers.collect().foreach(println(_))
// $example off$
spark.stop()
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala
index 0726fe9..a1e6b03 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala
@@ -89,8 +89,8 @@
vertexStorageLevel = vertexStorageLevel).cache()
val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_))
- println(s"GRAPHX: Number of vertices ${graph.vertices.count}")
- println(s"GRAPHX: Number of edges ${graph.edges.count}")
+ println(s"GRAPHX: Number of vertices ${graph.vertices.count()}")
+ println(s"GRAPHX: Number of edges ${graph.edges.count()}")
val pr = (numIterOpt match {
case Some(numIter) => PageRank.run(graph, numIter)
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala
index 6598863..314739a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala
@@ -39,7 +39,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession.
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
val sc = spark.sparkContext
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala
index 5377ddb..db99dde 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala
@@ -41,7 +41,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession.
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
val sc = spark.sparkContext
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala
index 9e9affc..772b9e3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala
@@ -34,7 +34,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession.
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
val sc = spark.sparkContext
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala
index 5e8b196..0a888a7 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala
@@ -36,7 +36,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession.
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
val sc = spark.sparkContext
@@ -60,7 +60,7 @@
},
(a, b) => math.min(a, b) // Merge Message
)
- println(sssp.vertices.collect.mkString("\n"))
+ println(sssp.vertices.collect().mkString("\n"))
// $example off$
spark.stop()
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala
index 8bc9c0a..6ba0371 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala
@@ -116,7 +116,7 @@
println(s"Total PageRank = $totalPR")
} else if (app == "cc") {
println("Running Connected Components")
- val numComponents = graph.connectedComponents.vertices.map(_._2).distinct().count()
+ val numComponents = graph.connectedComponents().vertices.map(_._2).distinct().count()
println(s"Number of components = $numComponents")
}
val runTime = System.currentTimeMillis() - startTime
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala
index b9bff69..1cb82f0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala
@@ -42,7 +42,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession.
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
val sc = spark.sparkContext
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala
index cdb33f4..999555f 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala
@@ -35,7 +35,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("AFTSurvivalRegressionExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala
index 354e65c..82b4136 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala
@@ -44,7 +44,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("ALSExample")
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala
index c2852aa..38e7eca 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala
@@ -26,7 +26,7 @@
object BinarizerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("BinarizerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala
index 14e13df..7dc4c5e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala
@@ -37,7 +37,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession
val spark = SparkSession
- .builder
+ .builder()
.appName("BisectingKMeansExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala
index 58f9fb3..b287a9d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala
@@ -34,7 +34,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession
val spark = SparkSession
- .builder
+ .builder()
.appName("BucketedRandomProjectionLSHExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala
index 7e65f9c..185a9f3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala
@@ -32,7 +32,7 @@
object BucketizerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("BucketizerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala
index 1a67a6e..7d23cbe 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala
@@ -27,7 +27,7 @@
object ChiSqSelectorExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("ChiSqSelectorExample")
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala
index 5146fd0..3748c50 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala
@@ -35,7 +35,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("ChiSquareTestExample")
.getOrCreate()
import spark.implicits._
@@ -51,7 +51,7 @@
)
val df = data.toDF("label", "features")
- val chi = ChiSquareTest.test(df, "features", "label").head
+ val chi = ChiSquareTest.test(df, "features", "label").head()
println(s"pValues = ${chi.getAs[Vector](0)}")
println(s"degreesOfFreedom ${chi.getSeq[Int](1).mkString("[", ",", "]")}")
println(s"statistics ${chi.getAs[Vector](2)}")
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CorrelationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CorrelationExample.scala
index d7f1fc8..994fe34 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/CorrelationExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/CorrelationExample.scala
@@ -36,7 +36,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("CorrelationExample")
.getOrCreate()
import spark.implicits._
@@ -50,10 +50,10 @@
)
val df = data.map(Tuple1.apply).toDF("features")
- val Row(coeff1: Matrix) = Correlation.corr(df, "features").head
+ val Row(coeff1: Matrix) = Correlation.corr(df, "features").head()
println(s"Pearson correlation matrix:\n $coeff1")
- val Row(coeff2: Matrix) = Correlation.corr(df, "features", "spearman").head
+ val Row(coeff2: Matrix) = Correlation.corr(df, "features", "spearman").head()
println(s"Spearman correlation matrix:\n $coeff2")
// $example off$
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala
index 947ca5f..e5921ae 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala
@@ -26,7 +26,7 @@
object CountVectorizerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("CountVectorizerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala
index 3383171..89fadd3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala
@@ -27,7 +27,7 @@
object DCTExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("DCTExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
index 4377efd..e0ec905 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
@@ -62,7 +62,7 @@
def run(params: Params): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"DataFrameExample with $params")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala
index 7f65fa3..e0c029e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala
@@ -30,7 +30,7 @@
object DecisionTreeClassificationExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("DecisionTreeClassificationExample")
.getOrCreate()
// $example on$
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala
index ef38163..9795e52 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala
@@ -166,7 +166,7 @@
algo: String,
fracTest: Double): (DataFrame, DataFrame) = {
val spark = SparkSession
- .builder
+ .builder()
.getOrCreate()
// Load training data
@@ -199,7 +199,7 @@
def run(params: Params): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"DecisionTreeExample with $params")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala
index aaaecae..39122c9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala
@@ -30,7 +30,7 @@
object DecisionTreeRegressionExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("DecisionTreeRegressionExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
index bfee330..46216e9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
@@ -39,7 +39,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("DeveloperApiExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala
index c0ffc01..91f1320 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala
@@ -27,7 +27,7 @@
object ElementwiseProductExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("ElementwiseProductExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala
index e5d91f1..56a4cd2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala
@@ -30,7 +30,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("EstimatorTransformerParamExample")
.getOrCreate()
@@ -58,7 +58,7 @@
// we can view the parameters it used during fit().
// This prints the parameter (name: value) pairs, where names are unique IDs for this
// LogisticRegression instance.
- println(s"Model 1 was fit using parameters: ${model1.parent.extractParamMap}")
+ println(s"Model 1 was fit using parameters: ${model1.parent.extractParamMap()}")
// We may alternatively specify parameters using a ParamMap,
// which supports several methods for specifying parameters.
@@ -73,7 +73,7 @@
// Now learn a new model using the paramMapCombined parameters.
// paramMapCombined overrides all parameters set earlier via lr.set* methods.
val model2 = lr.fit(training, paramMapCombined)
- println(s"Model 2 was fit using parameters: ${model2.parent.extractParamMap}")
+ println(s"Model 2 was fit using parameters: ${model2.parent.extractParamMap()}")
// Prepare test data.
val test = spark.createDataFrame(Seq(
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala
index 612a76fd..6b83224 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala
@@ -29,7 +29,7 @@
object FMClassifierExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("FMClassifierExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala
index 6bb06ea..0144188 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala
@@ -29,7 +29,7 @@
object FMRegressorExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("FMRegressorExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala
index bece0d9..b3c9a58 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala
@@ -33,7 +33,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FeatureHasherExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FeatureHasherExample.scala
index 1aed10b..a39cefc 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/FeatureHasherExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/FeatureHasherExample.scala
@@ -25,7 +25,7 @@
object FeatureHasherExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("FeatureHasherExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala
index ca4235d..5a9a526 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala
@@ -136,7 +136,7 @@
def run(params: Params): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"GBTExample with $params")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala
index 5e4bea4..bf8542d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala
@@ -34,7 +34,7 @@
object GaussianMixtureExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala
index 1b86d7c..5be5c72 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala
@@ -35,7 +35,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("GeneralizedLinearRegressionExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala
index a6d078d..1a0a86d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala
@@ -29,7 +29,7 @@
object GradientBoostedTreeClassifierExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("GradientBoostedTreeClassifierExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala
index 3feb234..d53f152 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala
@@ -29,7 +29,7 @@
object GradientBoostedTreeRegressorExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("GradientBoostedTreeRegressorExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ImputerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ImputerExample.scala
index 49e98d0..0d2a797 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/ImputerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/ImputerExample.scala
@@ -30,7 +30,7 @@
object ImputerExample {
def main(args: Array[String]): Unit = {
- val spark = SparkSession.builder
+ val spark = SparkSession.builder()
.appName("ImputerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala
index b3642c0..5b04fe5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala
@@ -27,7 +27,7 @@
object IndexToStringExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("IndexToStringExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/InteractionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/InteractionExample.scala
index 8113c99..6eed747 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/InteractionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/InteractionExample.scala
@@ -27,7 +27,7 @@
object InteractionExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("InteractionExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala
index 9bac16e..05f72c8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala
@@ -34,7 +34,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala
index 2bc8184..18ed687 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala
@@ -36,7 +36,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala
index 4215d37..8b83105 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala
@@ -34,7 +34,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala
index 370c6fd..bf21da9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala
@@ -102,7 +102,7 @@
def run(params: Params): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"LinearRegressionExample with $params")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala
index 4540a8d..566cf8c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala
@@ -27,7 +27,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("LinearRegressionWithElasticNetExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearSVCExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearSVCExample.scala
index 5f43e65..1ffdb10 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearSVCExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearSVCExample.scala
@@ -27,7 +27,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("LinearSVCExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala
index b64ab479..27f892b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala
@@ -110,7 +110,7 @@
def run(params: Params): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"LogisticRegressionExample with $params")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala
index 0368dcb..e7c8a93 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala
@@ -28,7 +28,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("LogisticRegressionSummaryExample")
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala
index 1847104..0d4fe3e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala
@@ -27,7 +27,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("LogisticRegressionWithElasticNetExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala
index 85d0713..7437308 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala
@@ -26,7 +26,7 @@
object MaxAbsScalerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("MaxAbsScalerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala
index 8515821..091c197 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala
@@ -34,7 +34,7 @@
def main(args: Array[String]): Unit = {
// Creates a SparkSession
val spark = SparkSession
- .builder
+ .builder()
.appName("MinHashLSHExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala
index 9ee6d9b..bc1502c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala
@@ -27,7 +27,7 @@
object MinMaxScalerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("MinMaxScalerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala
index 87d96dd..4c9be6c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala
@@ -42,7 +42,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("ModelSelectionViaCrossValidationExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala
index 71e41e7..43aab94 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala
@@ -36,7 +36,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("ModelSelectionViaTrainValidationSplitExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MulticlassLogisticRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MulticlassLogisticRegressionWithElasticNetExample.scala
index 1f7dbdd..4c0d145 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MulticlassLogisticRegressionWithElasticNetExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MulticlassLogisticRegressionWithElasticNetExample.scala
@@ -27,7 +27,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("MulticlassLogisticRegressionWithElasticNetExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala
index 646f46a..cc9c99b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala
@@ -31,7 +31,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("MultilayerPerceptronClassifierExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala
index d2183d6..0a11ba1 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala
@@ -26,7 +26,7 @@
object NGramExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("NGramExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala
index 50c70c6..3f8074e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala
@@ -27,7 +27,7 @@
object NaiveBayesExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("NaiveBayesExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala
index 989d250..8affc3b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala
@@ -27,7 +27,7 @@
object NormalizerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("NormalizerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala
index 742f3cd..e9c08fc 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala
@@ -26,7 +26,7 @@
object OneHotEncoderExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("OneHotEncoderExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala
index 86e70e8a..2b837fb 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala
@@ -36,7 +36,7 @@
object OneVsRestExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"OneVsRestExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala
index 4e1d7cd..4a0fd2b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala
@@ -27,7 +27,7 @@
object PCAExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("PCAExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala
index 12f8663..7556d82 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala
@@ -31,7 +31,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("PipelineExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala
index f117b03..2dbfe6c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala
@@ -27,7 +27,7 @@
object PolynomialExpansionExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("PolynomialExpansionExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PowerIterationClusteringExample.scala
index ca8f7af..34bb703 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/PowerIterationClusteringExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/PowerIterationClusteringExample.scala
@@ -26,7 +26,7 @@
object PowerIterationClusteringExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala
index b4e0811..e65b9e7 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala
@@ -33,7 +33,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala
index 55823fe..3e022d2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala
@@ -25,7 +25,7 @@
object QuantileDiscretizerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("QuantileDiscretizerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala
index 3498fa8..dc6cc7d68 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala
@@ -26,7 +26,7 @@
object RFormulaExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("RFormulaExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala
index 778a8ef..b4021cd 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala
@@ -29,7 +29,7 @@
object RandomForestClassifierExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("RandomForestClassifierExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala
index 6ba14bc..39ba1c7 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala
@@ -142,7 +142,7 @@
def run(params: Params): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName(s"RandomForestExample with $params")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala
index 2679fcb..2493dca 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala
@@ -29,7 +29,7 @@
object RandomForestRegressorExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("RandomForestRegressorExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RobustScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RobustScalerExample.scala
index 4f40c90..f0cf0af 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/RobustScalerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/RobustScalerExample.scala
@@ -26,7 +26,7 @@
object RobustScalerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("RobustScalerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala
index bf6a484..1137504 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala
@@ -26,7 +26,7 @@
object SQLTransformerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("SQLTransformerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala
index 4d668e8..77a63c5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala
@@ -26,7 +26,7 @@
object StandardScalerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("StandardScalerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala
index 369a6ff..510a806 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala
@@ -26,7 +26,7 @@
object StopWordsRemoverExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("StopWordsRemoverExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala
index 63f273e..bcfd1a8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala
@@ -26,7 +26,7 @@
object StringIndexerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("StringIndexerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SummarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SummarizerExample.scala
index 2f54d1d..552f686 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SummarizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SummarizerExample.scala
@@ -27,7 +27,7 @@
object SummarizerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("SummarizerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala
index 6121c81..c3c9df5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala
@@ -27,7 +27,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("TfIdfExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala
index 1547776..b29e965 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala
@@ -27,7 +27,7 @@
object TokenizerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("TokenizerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/UnivariateFeatureSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/UnivariateFeatureSelectorExample.scala
index e4932db..4288bd5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/UnivariateFeatureSelectorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/UnivariateFeatureSelectorExample.scala
@@ -34,7 +34,7 @@
object UnivariateFeatureSelectorExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("UnivariateFeatureSelectorExample")
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VarianceThresholdSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VarianceThresholdSelectorExample.scala
index e418526..6a40bb3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/VarianceThresholdSelectorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/VarianceThresholdSelectorExample.scala
@@ -34,7 +34,7 @@
object VarianceThresholdSelectorExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("VarianceThresholdSelectorExample")
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala
index 3d5c7ef..b7b2f5c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala
@@ -27,7 +27,7 @@
object VectorAssemblerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("VectorAssemblerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala
index 96bb8ea..222b17d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala
@@ -26,7 +26,7 @@
object VectorIndexerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("VectorIndexerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSizeHintExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSizeHintExample.scala
index 688731a..b2e562a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSizeHintExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSizeHintExample.scala
@@ -27,7 +27,7 @@
object VectorSizeHintExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("VectorSizeHintExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala
index 9a0af5d..1272782 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala
@@ -31,7 +31,7 @@
object VectorSlicerExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("VectorSlicerExample")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala
index 8ff0e8c..42f6fb0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala
@@ -28,7 +28,7 @@
object Word2VecExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("Word2Vec example")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala
index a606cc4..3f9ec1a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala
@@ -46,7 +46,7 @@
.run(training)
// Clear the prediction threshold so the model will return probabilities
- model.clearThreshold
+ model.clearThreshold()
// Compute raw scores on the test set
val predictionAndLabels = test.map { case LabeledPoint(label, features) =>
@@ -58,44 +58,44 @@
val metrics = new BinaryClassificationMetrics(predictionAndLabels)
// Precision by threshold
- val precision = metrics.precisionByThreshold
- precision.collect.foreach { case (t, p) =>
+ val precision = metrics.precisionByThreshold()
+ precision.collect().foreach { case (t, p) =>
println(s"Threshold: $t, Precision: $p")
}
// Recall by threshold
- val recall = metrics.recallByThreshold
- recall.collect.foreach { case (t, r) =>
+ val recall = metrics.recallByThreshold()
+ recall.collect().foreach { case (t, r) =>
println(s"Threshold: $t, Recall: $r")
}
// Precision-Recall Curve
- val PRC = metrics.pr
+ val PRC = metrics.pr()
// F-measure
- val f1Score = metrics.fMeasureByThreshold
- f1Score.collect.foreach { case (t, f) =>
+ val f1Score = metrics.fMeasureByThreshold()
+ f1Score.collect().foreach { case (t, f) =>
println(s"Threshold: $t, F-score: $f, Beta = 1")
}
val beta = 0.5
val fScore = metrics.fMeasureByThreshold(beta)
- fScore.collect.foreach { case (t, f) =>
+ fScore.collect().foreach { case (t, f) =>
println(s"Threshold: $t, F-score: $f, Beta = 0.5")
}
// AUPRC
- val auPRC = metrics.areaUnderPR
+ val auPRC = metrics.areaUnderPR()
println(s"Area under precision-recall curve = $auPRC")
// Compute thresholds used in ROC and PR curves
val thresholds = precision.map(_._1)
// ROC Curve
- val roc = metrics.roc
+ val roc = metrics.roc()
// AUROC
- val auROC = metrics.areaUnderROC
+ val auROC = metrics.areaUnderROC()
println(s"Area under ROC = $auROC")
// $example off$
sc.stop()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala
index 6ed59a3..d68443d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala
@@ -53,7 +53,7 @@
// $example off$
println("filtered data: ")
- filteredData.collect.foreach(x => println(x))
+ filteredData.collect().foreach(x => println(x))
sc.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala
index d6ec678..e69e0dc 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala
@@ -45,10 +45,10 @@
// $example off$
println("transformedData: ")
- transformedData.collect.foreach(x => println(x))
+ transformedData.collect().foreach(x => println(x))
println("transformedData2: ")
- transformedData2.collect.foreach(x => println(x))
+ transformedData2.collect().foreach(x => println(x))
sc.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala
index 3c56e19..ebfd772 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala
@@ -53,7 +53,7 @@
val prediction = model.predict(point.features)
(point.label, prediction)
}
- val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count()
+ val testErr = labelAndPreds.filter(r => r._1 != r._2).count().toDouble / testData.count()
println(s"Test Error = $testErr")
println(s"Learned classification GBT model:\n ${model.toDebugString}")
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
index d80f54d..c00f89b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
@@ -193,7 +193,7 @@
stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = {
val spark = SparkSession
- .builder
+ .builder()
.sparkContext(sc)
.getOrCreate()
import spark.implicits._
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala
index 55a45b3..e3465f1 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala
@@ -36,7 +36,7 @@
val data = sc.textFile("data/mllib/sample_lda_data.txt")
val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble)))
// Index documents with unique IDs
- val corpus = parsedData.zipWithIndex.map(_.swap).cache()
+ val corpus = parsedData.zipWithIndex().map(_.swap).cache()
// Cluster the documents into three topics using LDA
val ldaModel = new LDA().setK(3).run(corpus)
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala
index b1cad7b..2b33a7e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala
@@ -46,10 +46,10 @@
// $example off$
println("data1: ")
- data1.collect.foreach(x => println(x))
+ data1.collect().foreach(x => println(x))
println("data2: ")
- data2.collect.foreach(x => println(x))
+ data2.collect().foreach(x => println(x))
sc.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala
index 96deafd..3ba8eea 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala
@@ -41,7 +41,7 @@
val clusters = KMeans.train(parsedData, numClusters, numIterations)
// Export to PMML to a String in PMML format
- println(s"PMML Model:\n ${clusters.toPMML}")
+ println(s"PMML Model:\n ${clusters.toPMML()}")
// Export the model to a local file in PMML format
clusters.toPMML("/tmp/kmeans.xml")
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala
index 246e71d..ce2b9ab 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala
@@ -54,7 +54,7 @@
val prediction = model.predict(point.features)
(point.label, prediction)
}
- val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count()
+ val testErr = labelAndPreds.filter(r => r._1 != r._2).count().toDouble / testData.count()
println(s"Test Error = $testErr")
println(s"Learned classification forest model:\n ${model.toDebugString}")
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala
index 7a7501e..ece6afb 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala
@@ -27,7 +27,7 @@
object RankingMetricsExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.appName("RankingMetricsExample")
.getOrCreate()
// $example on$
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala
index 66a608c..38a8be4 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala
@@ -49,10 +49,10 @@
// $example off$
println("data1: ")
- data1.collect.foreach(x => println(x))
+ data1.collect().foreach(x => println(x))
println("data2: ")
- data2.collect.foreach(x => println(x))
+ data2.collect().foreach(x => println(x))
sc.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala
index 14b2a20..dae1f4c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala
@@ -55,10 +55,10 @@
// $example off$
println("tfidf: ")
- tfidf.collect.foreach(x => println(x))
+ tfidf.collect().foreach(x => println(x))
println("tfidfIgnore: ")
- tfidfIgnore.collect.foreach(x => println(x))
+ tfidfIgnore.collect().foreach(x => println(x))
sc.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
index 4fd482d..9c33e87 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
@@ -31,7 +31,7 @@
def main(args: Array[String]): Unit = {
// $example on:init_session$
val spark = SparkSession
- .builder
+ .builder()
.appName("Spark Examples")
.config("spark.some.config.option", "some-value")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SimpleTypedAggregator.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SimpleTypedAggregator.scala
index 5d11fb2..7aabd18 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/SimpleTypedAggregator.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/SimpleTypedAggregator.scala
@@ -25,7 +25,7 @@
def main(args: Array[String]): Unit = {
val spark = SparkSession
- .builder
+ .builder()
.master("local")
.appName("common typed aggregator implementations")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredComplexSessionization.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredComplexSessionization.scala
index f25ba29..4b228de 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredComplexSessionization.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredComplexSessionization.scala
@@ -102,7 +102,7 @@
val port = args(1).toInt
val spark = SparkSession
- .builder
+ .builder()
.appName("StructuredComplexSessionization")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKafkaWordCount.scala
index 2aab49c..b2755f8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKafkaWordCount.scala
@@ -60,7 +60,7 @@
if (args.length > 3) args(3) else "/tmp/temporary-" + UUID.randomUUID.toString
val spark = SparkSession
- .builder
+ .builder()
.appName("StructuredKafkaWordCount")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKerberizedKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKerberizedKafkaWordCount.scala
index 1f7d2c7..013a3c8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKerberizedKafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKerberizedKafkaWordCount.scala
@@ -99,7 +99,7 @@
if (args.length > 3) args(3) else "/tmp/temporary-" + UUID.randomUUID.toString
val spark = SparkSession
- .builder
+ .builder()
.appName("StructuredKerberizedKafkaWordCount")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala
index 6dbc70b..3e14e14 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala
@@ -44,7 +44,7 @@
val port = args(1).toInt
val spark = SparkSession
- .builder
+ .builder()
.appName("StructuredNetworkWordCount")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala
index 4ba2c6b..8301e92 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala
@@ -66,7 +66,7 @@
val slideDuration = s"$slideSize seconds"
val spark = SparkSession
- .builder
+ .builder()
.appName("StructuredNetworkWordCountWindowed")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala
index 5d99738..14af172 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala
@@ -47,7 +47,7 @@
val port = args(1).toInt
val spark = SparkSession
- .builder
+ .builder()
.appName("StructuredSessionization")
.getOrCreate()
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
index 626f4b4..c88a907 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
@@ -87,7 +87,7 @@
val reader = new BufferedReader(
new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8))
userInput = reader.readLine()
- while(!isStopped && userInput != null) {
+ while(!isStopped() && userInput != null) {
store(userInput)
userInput = reader.readLine()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
index 7daa001..64da35f 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
@@ -94,7 +94,7 @@
def getInstance(sparkConf: SparkConf): SparkSession = {
if (instance == null) {
instance = SparkSession
- .builder
+ .builder()
.config(sparkConf)
.getOrCreate()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
index 8a5fcda..05f111b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
@@ -60,7 +60,7 @@
// Update the cumulative count using mapWithState
// This will give a DStream made of state (which is the cumulative count of the words)
val mappingFunc = (word: String, one: Option[Int], state: State[Int]) => {
- val sum = one.getOrElse(0) + state.getOption.getOrElse(0)
+ val sum = one.getOrElse(0) + state.getOption().getOrElse(0)
val output = (word, sum)
state.update(sum)
output
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index a07c2e1..ab40fcf 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -277,7 +277,7 @@
if (Random.nextDouble() < probability) { Some(vidVvals._1) }
else { None }
}
- if (selectedVertices.count > 0) {
+ if (selectedVertices.count() > 0) {
found = true
val collectedVertices = selectedVertices.collect()
retVal = collectedVertices(Random.nextInt(collectedVertices.length))
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 4a79087..21fb284 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -321,7 +321,7 @@
// Convert the vertex partitions in edges to the correct type
val newEdges = edges.asInstanceOf[EdgeRDDImpl[ED, _]]
- .mapEdgePartitions((pid, part) => part.withoutVertexAttributes[VD])
+ .mapEdgePartitions((pid, part) => part.withoutVertexAttributes[VD]())
.cache()
GraphImpl.fromExistingRDDs(vertices, newEdges)
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala
index 8fd3e6f..bc23714 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala
@@ -63,11 +63,11 @@
test("count") {
withSpark { sc =>
val empty = EdgeRDD.fromEdges(sc.emptyRDD[Edge[Int]])
- assert(empty.count === 0)
+ assert(empty.count() === 0)
val edges = List(Edge(0, 1, ()), Edge(1, 2, ()), Edge(2, 0, ()))
val nonempty = EdgeRDD.fromEdges(sc.parallelize(edges))
- assert(nonempty.count === edges.size)
+ assert(nonempty.count() === edges.size)
}
}
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala
index e55b05f..60c5621 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala
@@ -39,7 +39,7 @@
val neighborAttrSums = graph.aggregateMessages[Int](
ctx => ctx.sendToDst(ctx.srcAttr),
_ + _)
- assert(neighborAttrSums.collect.toSet === Set((0: VertexId, 100)))
+ assert(neighborAttrSums.collect().toSet === Set((0: VertexId, 100)))
} finally {
Utils.deleteRecursively(tmpDir)
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
index 3298171..3d4dde9 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
@@ -41,10 +41,10 @@
withSpark { sc =>
val graph = getCycleGraph(sc, 100)
val nbrs = graph.collectNeighborIds(EdgeDirection.Either).cache()
- assert(nbrs.count === 100)
- assert(graph.numVertices === nbrs.count)
- nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) }
- nbrs.collect.foreach {
+ assert(nbrs.count() === 100)
+ assert(graph.numVertices === nbrs.count())
+ nbrs.collect().foreach { case (vid, nbrs) => assert(nbrs.size === 2) }
+ nbrs.collect().foreach {
case (vid, nbrs) =>
val s = nbrs.toSet
assert(s.contains((vid + 1) % 100))
@@ -62,7 +62,7 @@
val correctEdges = edgeArray.filter { case (a, b) => a != b }.toSet
val graph = Graph.fromEdgeTuples(sc.parallelize(edgeArray), 1)
val canonicalizedEdges = graph.removeSelfEdges().edges.map(e => (e.srcId, e.dstId))
- .collect
+ .collect()
assert(canonicalizedEdges.toSet.size === canonicalizedEdges.size)
assert(canonicalizedEdges.toSet === correctEdges)
}
@@ -110,9 +110,9 @@
withSpark { sc =>
val graph = getCycleGraph(sc, 100)
val edges = graph.collectEdges(EdgeDirection.Out).cache()
- assert(edges.count == 100)
- edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) }
- edges.collect.foreach {
+ assert(edges.count() == 100)
+ edges.collect().foreach { case (vid, edges) => assert(edges.size == 1) }
+ edges.collect().foreach {
case (vid, edges) =>
val s = edges.toSet
val edgeDstIds = s.map(e => e.dstId)
@@ -125,9 +125,9 @@
withSpark { sc =>
val graph = getCycleGraph(sc, 100)
val edges = graph.collectEdges(EdgeDirection.In).cache()
- assert(edges.count == 100)
- edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) }
- edges.collect.foreach {
+ assert(edges.count() == 100)
+ edges.collect().foreach { case (vid, edges) => assert(edges.size == 1) }
+ edges.collect().foreach {
case (vid, edges) =>
val s = edges.toSet
val edgeSrcIds = s.map(e => e.srcId)
@@ -140,9 +140,9 @@
withSpark { sc =>
val graph = getCycleGraph(sc, 100)
val edges = graph.collectEdges(EdgeDirection.Either).cache()
- assert(edges.count == 100)
- edges.collect.foreach { case (vid, edges) => assert(edges.size == 2) }
- edges.collect.foreach {
+ assert(edges.count() == 100)
+ edges.collect().foreach { case (vid, edges) => assert(edges.size == 2) }
+ edges.collect().foreach {
case (vid, edges) =>
val s = edges.toSet
val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId)
@@ -156,9 +156,9 @@
withSpark { sc =>
val graph = getChainGraph(sc, 50)
val edges = graph.collectEdges(EdgeDirection.Out).cache()
- assert(edges.count == 49)
- edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) }
- edges.collect.foreach {
+ assert(edges.count() == 49)
+ edges.collect().foreach { case (vid, edges) => assert(edges.size == 1) }
+ edges.collect().foreach {
case (vid, edges) =>
val s = edges.toSet
val edgeDstIds = s.map(e => e.dstId)
@@ -173,9 +173,9 @@
val edges = graph.collectEdges(EdgeDirection.In).cache()
// We expect only 49 because collectEdges does not return vertices that do
// not have any edges in the specified direction.
- assert(edges.count == 49)
- edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) }
- edges.collect.foreach {
+ assert(edges.count() == 49)
+ edges.collect().foreach { case (vid, edges) => assert(edges.size == 1) }
+ edges.collect().foreach {
case (vid, edges) =>
val s = edges.toSet
val edgeDstIds = s.map(e => e.srcId)
@@ -190,15 +190,15 @@
val edges = graph.collectEdges(EdgeDirection.Either).cache()
// We expect only 49 because collectEdges does not return vertices that do
// not have any edges in the specified direction.
- assert(edges.count === 50)
- edges.collect.foreach {
+ assert(edges.count() === 50)
+ edges.collect().foreach {
case (vid, edges) => if (vid > 0 && vid < 49) {
assert(edges.size == 2)
} else {
assert(edges.size == 1)
}
}
- edges.collect.foreach {
+ edges.collect().foreach {
case (vid, edges) =>
val s = edges.toSet
val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId)
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
index 459cddb..a0f2c32 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
@@ -94,18 +94,18 @@
// The two edges start out in different partitions
for (edges <- List(identicalEdges, canonicalEdges, sameSrcEdges)) {
- assert(nonemptyParts(mkGraph(edges)).count === 2)
+ assert(nonemptyParts(mkGraph(edges)).count() === 2)
}
// partitionBy(RandomVertexCut) puts identical edges in the same partition
- assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(RandomVertexCut)).count === 1)
+ assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(RandomVertexCut)).count() === 1)
// partitionBy(EdgePartition1D) puts same-source edges in the same partition
- assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count === 1)
+ assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count() === 1)
// partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into
// the same partition
assert(
- nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1)
+ nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count() === 1)
// partitionBy(EdgePartition2D) puts identical edges in the same partition
- assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count === 1)
+ assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count() === 1)
// partitionBy(EdgePartition2D) ensures that vertices need only be replicated to 2 * sqrt(p)
// partitions
@@ -122,7 +122,7 @@
val partitionSets = partitionedGraph.edges.partitionsRDD.mapPartitions { iter =>
val part = iter.next()._2
Iterator((part.iterator.flatMap(e => Iterator(e.srcId, e.dstId))).toSet)
- }.collect
+ }.collect()
if (!verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) {
val numFailures = verts.count(id => partitionSets.count(_.contains(id)) > bound)
val failure = verts.maxBy(id => partitionSets.count(_.contains(id)))
@@ -134,7 +134,7 @@
val partitionSetsUnpartitioned = graph.edges.partitionsRDD.mapPartitions { iter =>
val part = iter.next()._2
Iterator((part.iterator.flatMap(e => Iterator(e.srcId, e.dstId))).toSet)
- }.collect
+ }.collect()
assert(verts.exists(id => partitionSetsUnpartitioned.count(_.contains(id)) > bound))
// Forming triplets view
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
index 90a9ac6..c8e50e5 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
@@ -30,7 +30,7 @@
(vid, attr, msg) => attr,
et => Iterator.empty,
(a: Int, b: Int) => throw new Exception("mergeMsg run unexpectedly"))
- assert(result.vertices.collect.toSet === star.vertices.collect.toSet)
+ assert(result.vertices.collect().toSet === star.vertices.collect().toSet)
}
}
@@ -40,16 +40,16 @@
val chain = Graph.fromEdgeTuples(
sc.parallelize((1 until n).map(x => (x: VertexId, x + 1: VertexId)), 3),
0).cache()
- assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexId, 0)).toSet)
+ assert(chain.vertices.collect().toSet === (1 to n).map(x => (x: VertexId, 0)).toSet)
val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 }.cache()
- assert(chainWithSeed.vertices.collect.toSet ===
+ assert(chainWithSeed.vertices.collect().toSet ===
Set((1: VertexId, 1)) ++ (2 to n).map(x => (x: VertexId, 0)).toSet)
val result = Pregel(chainWithSeed, 0)(
(vid, attr, msg) => math.max(msg, attr),
et => if (et.dstAttr != et.srcAttr) Iterator((et.dstId, et.srcAttr)) else Iterator.empty,
(a: Int, b: Int) => math.max(a, b))
- assert(result.vertices.collect.toSet ===
- chain.vertices.mapValues { (vid, attr) => attr + 1 }.collect.toSet)
+ assert(result.vertices.collect().toSet ===
+ chain.vertices.mapValues { (vid, attr) => attr + 1 }.collect().toSet)
}
}
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
index 434e6a8..0a5dcf8 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
@@ -33,7 +33,7 @@
val n = 100
val verts = vertices(sc, n)
val evens = verts.filter(q => ((q._2 % 2) == 0))
- assert(evens.count === (0 to n).count(_ % 2 == 0))
+ assert(evens.count() === (0 to n).count(_ % 2 == 0))
}
}
@@ -42,7 +42,7 @@
val n = 100
val verts = vertices(sc, n)
val negatives = verts.mapValues(x => -x).cache() // Allow joining b with a derived RDD of b
- assert(negatives.count === n + 1)
+ assert(negatives.count() === n + 1)
}
}
@@ -227,11 +227,11 @@
test("count") {
withSpark { sc =>
val empty = VertexRDD(sc.emptyRDD[(Long, Unit)])
- assert(empty.count === 0)
+ assert(empty.count() === 0)
val n = 100
val nonempty = vertices(sc, n)
- assert(nonempty.count === n + 1)
+ assert(nonempty.count() === n + 1)
}
}
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala
index 808877f..95f85d9 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala
@@ -33,9 +33,9 @@
val labels = LabelPropagation.run(graph, n * 4).cache()
// All vertices within a clique should have the same label
- val clique1Labels = labels.vertices.filter(_._1 < n).map(_._2).collect.toArray
+ val clique1Labels = labels.vertices.filter(_._1 < n).map(_._2).collect().toArray
assert(clique1Labels.forall(_ == clique1Labels(0)))
- val clique2Labels = labels.vertices.filter(_._1 >= n).map(_._2).collect.toArray
+ val clique2Labels = labels.vertices.filter(_._1 >= n).map(_._2).collect().toArray
assert(clique2Labels.forall(_ == clique2Labels(0)))
// The two cliques should have different labels
assert(clique1Labels(0) != clique2Labels(0))
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
index da0457c..da139ce 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
@@ -45,8 +45,8 @@
val edges = sc.emptyRDD[Edge[Double]]
val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations
val (graph, _) = SVDPlusPlus.run(edges, conf)
- assert(graph.vertices.count == 0)
- assert(graph.edges.count == 0)
+ assert(graph.vertices.count() == 0)
+ assert(graph.edges.count() == 0)
}
}
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
index f909b70..527187d 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
@@ -33,7 +33,7 @@
val edges = sc.parallelize(edgeSeq).map { case (v1, v2) => (v1.toLong, v2.toLong) }
val graph = Graph.fromEdgeTuples(edges, 1)
val landmarks = Seq(1, 4).map(_.toLong)
- val results = ShortestPaths.run(graph, landmarks).vertices.collect.map {
+ val results = ShortestPaths.run(graph, landmarks).vertices.collect().map {
case (v, spMap) => (v, spMap.mapValues(i => i).toMap)
}
assert(results.toSet === shortestPaths)
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
index abbd89b..2454e9b 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
@@ -30,7 +30,7 @@
val graph = Graph.fromEdgeTuples(rawEdges, true).cache()
val triangleCount = graph.triangleCount()
val verts = triangleCount.vertices
- verts.collect.foreach { case (vid, count) => assert(count === 1) }
+ verts.collect().foreach { case (vid, count) => assert(count === 1) }
}
}
@@ -80,7 +80,7 @@
val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache()
val triangleCount = graph.triangleCount()
val verts = triangleCount.vertices
- verts.collect.foreach { case (vid, count) => assert(count === 1) }
+ verts.collect().foreach { case (vid, count) => assert(count === 1) }
}
}
diff --git a/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/CommitterBindingSuite.scala b/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/CommitterBindingSuite.scala
index 984c7db..2a2d25a 100644
--- a/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/CommitterBindingSuite.scala
+++ b/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/CommitterBindingSuite.scala
@@ -62,7 +62,7 @@
StubPathOutputCommitterBinding.bindWithDynamicPartitioning(conf, "http")
val tContext: TaskAttemptContext = new TaskAttemptContextImpl(conf, taskAttemptId0)
val parquet = new BindingParquetOutputCommitter(path, tContext)
- val inner = parquet.boundCommitter.asInstanceOf[StubPathOutputCommitterWithDynamicPartioning]
+ val inner = parquet.boundCommitter().asInstanceOf[StubPathOutputCommitterWithDynamicPartioning]
parquet.setupJob(tContext)
assert(inner.jobSetup, s"$inner job not setup")
parquet.setupTask(tContext)
diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala
index 827ca3f..016a836 100644
--- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala
+++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala
@@ -806,7 +806,7 @@
s += 1
}
}
- new SparseVector(ns, indexBuff.result, valueBuff.result)
+ new SparseVector(ns, indexBuff.result(), valueBuff.result())
}
private[spark] override def iterator: Iterator[(Int, Double)] = {
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 6d98f60..81c43f7 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
@@ -56,7 +56,7 @@
val iters = 1e2.toInt
val rnd = new scala.util.Random(0)
- val f2jBLAS = getF2jBLAS.getOrElse(throw new RuntimeException("can't load F2jBLAS"))
+ val f2jBLAS = getF2jBLAS().getOrElse(throw new RuntimeException("can't load F2jBLAS"))
val javaBLAS = BLAS.javaBLAS
val nativeBLAS = BLAS.nativeBLAS
@@ -91,9 +91,9 @@
runBenchmark("daxpy") {
val n = 1e8.toInt
- val alpha = rnd.nextDouble
- val x = Array.fill(n) { rnd.nextDouble }
- val y = Array.fill(n) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val x = Array.fill(n) { rnd.nextDouble() }
+ val y = Array.fill(n) { rnd.nextDouble() }
runBLASBenchmark("daxpy", n) { impl =>
impl.daxpy(n, alpha, x, 1, y.clone, 1)
@@ -102,9 +102,9 @@
runBenchmark("saxpy") {
val n = 1e8.toInt
- val alpha = rnd.nextFloat
- val x = Array.fill(n) { rnd.nextFloat }
- val y = Array.fill(n) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val x = Array.fill(n) { rnd.nextFloat() }
+ val y = Array.fill(n) { rnd.nextFloat() }
runBLASBenchmark("saxpy", n) { impl =>
impl.saxpy(n, alpha, x, 1, y.clone, 1)
@@ -113,7 +113,7 @@
runBenchmark("dcopy") {
val n = 1e8.toInt
- val x = Array.fill(n) { rnd.nextDouble }
+ val x = Array.fill(n) { rnd.nextDouble() }
val y = Array.fill(n) { 0.0 }
runBLASBenchmark("dcopy", n) { impl =>
@@ -123,7 +123,7 @@
runBenchmark("scopy") {
val n = 1e8.toInt
- val x = Array.fill(n) { rnd.nextFloat }
+ val x = Array.fill(n) { rnd.nextFloat() }
val y = Array.fill(n) { 0.0f }
runBLASBenchmark("scopy", n) { impl =>
@@ -133,8 +133,8 @@
runBenchmark("ddot") {
val n = 1e8.toInt
- val x = Array.fill(n) { rnd.nextDouble }
- val y = Array.fill(n) { rnd.nextDouble }
+ val x = Array.fill(n) { rnd.nextDouble() }
+ val y = Array.fill(n) { rnd.nextDouble() }
runBLASBenchmark("ddot", n) { impl =>
impl.ddot(n, x, 1, y, 1)
@@ -143,8 +143,8 @@
runBenchmark("sdot") {
val n = 1e8.toInt
- val x = Array.fill(n) { rnd.nextFloat }
- val y = Array.fill(n) { rnd.nextFloat }
+ val x = Array.fill(n) { rnd.nextFloat() }
+ val y = Array.fill(n) { rnd.nextFloat() }
runBLASBenchmark("sdot", n) { impl =>
impl.sdot(n, x, 1, y, 1)
@@ -153,7 +153,7 @@
runBenchmark("dnrm2") {
val n = 1e8.toInt
- val x = Array.fill(n) { rnd.nextDouble }
+ val x = Array.fill(n) { rnd.nextDouble() }
runBLASBenchmark("dnrm2", n) { impl =>
impl.dnrm2(n, x, 1)
@@ -162,7 +162,7 @@
runBenchmark("snrm2") {
val n = 1e8.toInt
- val x = Array.fill(n) { rnd.nextFloat }
+ val x = Array.fill(n) { rnd.nextFloat() }
runBLASBenchmark("snrm2", n) { impl =>
impl.snrm2(n, x, 1)
@@ -171,8 +171,8 @@
runBenchmark("dscal") {
val n = 1e8.toInt
- val alpha = rnd.nextDouble
- val x = Array.fill(n) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val x = Array.fill(n) { rnd.nextDouble() }
runBLASBenchmark("dscal", n) { impl =>
impl.dscal(n, alpha, x.clone, 1)
@@ -181,8 +181,8 @@
runBenchmark("sscal") {
val n = 1e8.toInt
- val alpha = rnd.nextFloat
- val x = Array.fill(n) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val x = Array.fill(n) { rnd.nextFloat() }
runBLASBenchmark("sscal", n) { impl =>
impl.sscal(n, alpha, x.clone, 1)
@@ -192,12 +192,12 @@
runBenchmark("dgemv[N]") {
val m = 1e4.toInt
val n = 1e4.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(m * n) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(m * n) { rnd.nextDouble() }
val lda = m
- val x = Array.fill(n) { rnd.nextDouble }
- val beta = rnd.nextDouble
- val y = Array.fill(m) { rnd.nextDouble }
+ val x = Array.fill(n) { rnd.nextDouble() }
+ val beta = rnd.nextDouble()
+ val y = Array.fill(m) { rnd.nextDouble() }
runBLASBenchmark("dgemv[N]", m * n) { impl =>
impl.dgemv("N", m, n, alpha, a, lda, x, 1, beta, y.clone, 1)
@@ -207,12 +207,12 @@
runBenchmark("dgemv[T]") {
val m = 1e4.toInt
val n = 1e4.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(m * n) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(m * n) { rnd.nextDouble() }
val lda = m
- val x = Array.fill(m) { rnd.nextDouble }
- val beta = rnd.nextDouble
- val y = Array.fill(n) { rnd.nextDouble }
+ val x = Array.fill(m) { rnd.nextDouble() }
+ val beta = rnd.nextDouble()
+ val y = Array.fill(n) { rnd.nextDouble() }
runBLASBenchmark("dgemv[T]", m * n) { impl =>
impl.dgemv("T", m, n, alpha, a, lda, x, 1, beta, y.clone, 1)
@@ -222,12 +222,12 @@
runBenchmark("sgemv[N]") {
val m = 1e4.toInt
val n = 1e4.toInt
- val alpha = rnd.nextFloat
- val a = Array.fill(m * n) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val a = Array.fill(m * n) { rnd.nextFloat() }
val lda = m
- val x = Array.fill(n) { rnd.nextFloat }
- val beta = rnd.nextFloat
- val y = Array.fill(m) { rnd.nextFloat }
+ val x = Array.fill(n) { rnd.nextFloat() }
+ val beta = rnd.nextFloat()
+ val y = Array.fill(m) { rnd.nextFloat() }
runBLASBenchmark("sgemv[N]", m * n) { impl =>
impl.sgemv("N", m, n, alpha, a, lda, x, 1, beta, y.clone, 1)
@@ -237,12 +237,12 @@
runBenchmark("sgemv[T]") {
val m = 1e4.toInt
val n = 1e4.toInt
- val alpha = rnd.nextFloat
- val a = Array.fill(m * n) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val a = Array.fill(m * n) { rnd.nextFloat() }
val lda = m
- val x = Array.fill(m) { rnd.nextFloat }
- val beta = rnd.nextFloat
- val y = Array.fill(n) { rnd.nextFloat }
+ val x = Array.fill(m) { rnd.nextFloat() }
+ val beta = rnd.nextFloat()
+ val y = Array.fill(n) { rnd.nextFloat() }
runBLASBenchmark("sgemv[T]", m * n) { impl =>
impl.sgemv("T", m, n, alpha, a, lda, x, 1, beta, y.clone, 1)
@@ -252,12 +252,12 @@
runBenchmark("dger") {
val m = 1e4.toInt
val n = 1e4.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(m * n) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(m * n) { rnd.nextDouble() }
val lda = m
- val x = Array.fill(n) { rnd.nextDouble }
- val beta = rnd.nextDouble
- val y = Array.fill(m) { rnd.nextDouble }
+ val x = Array.fill(n) { rnd.nextDouble() }
+ val beta = rnd.nextDouble()
+ val y = Array.fill(m) { rnd.nextDouble() }
runBLASBenchmark("dger", m * n) { impl =>
impl.dger(m, n, alpha, x, 1, y, 1, a.clone(), m)
@@ -266,11 +266,11 @@
runBenchmark("dspmv[U]") {
val n = 1e4.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(n * (n + 1) / 2) { rnd.nextDouble }
- val x = Array.fill(n) { rnd.nextDouble }
- val beta = rnd.nextDouble
- val y = Array.fill(n) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(n * (n + 1) / 2) { rnd.nextDouble() }
+ val x = Array.fill(n) { rnd.nextDouble() }
+ val beta = rnd.nextDouble()
+ val y = Array.fill(n) { rnd.nextDouble() }
runBLASBenchmark("dspmv[U]", n * (n + 1) / 2) { impl =>
impl.dspmv("U", n, alpha, a, x, 1, beta, y.clone, 1)
@@ -279,9 +279,9 @@
runBenchmark("dspr[U]") {
val n = 1e4.toInt
- val alpha = rnd.nextDouble
- val x = Array.fill(n) { rnd.nextDouble }
- val a = Array.fill(n * (n + 1) / 2) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val x = Array.fill(n) { rnd.nextDouble() }
+ val a = Array.fill(n * (n + 1) / 2) { rnd.nextDouble() }
runBLASBenchmark("dspr[U]", n * (n + 1) / 2) { impl =>
impl.dspr("U", n, alpha, x, 1, a.clone)
@@ -290,9 +290,9 @@
runBenchmark("dsyr[U]") {
val n = 1e4.toInt
- val alpha = rnd.nextDouble
- val x = Array.fill(n) { rnd.nextDouble }
- val a = Array.fill(n * n) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val x = Array.fill(n) { rnd.nextDouble() }
+ val a = Array.fill(n * n) { rnd.nextDouble() }
runBLASBenchmark("dsyr[U]", n * (n + 1) / 2) { impl =>
impl.dsyr("U", n, alpha, x, 1, a.clone, n)
@@ -303,13 +303,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(m * k) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(m * k) { rnd.nextDouble() }
val lda = m
- val b = Array.fill(k * n) { rnd.nextDouble }
+ val b = Array.fill(k * n) { rnd.nextDouble() }
val ldb = k
- val beta = rnd.nextDouble
- val c = Array.fill(m * n) { rnd.nextDouble }
+ val beta = rnd.nextDouble()
+ val c = Array.fill(m * n) { rnd.nextDouble() }
val ldc = m
runBLASBenchmark("dgemm[N,N]", m * n * k) { impl =>
@@ -321,13 +321,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(m * k) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(m * k) { rnd.nextDouble() }
val lda = m
- val b = Array.fill(k * n) { rnd.nextDouble }
+ val b = Array.fill(k * n) { rnd.nextDouble() }
val ldb = n
- val beta = rnd.nextDouble
- val c = Array.fill(m * n) { rnd.nextDouble }
+ val beta = rnd.nextDouble()
+ val c = Array.fill(m * n) { rnd.nextDouble() }
val ldc = m
runBLASBenchmark("dgemm[N,T]", m * n * k) { impl =>
@@ -339,13 +339,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(m * k) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(m * k) { rnd.nextDouble() }
val lda = k
- val b = Array.fill(k * n) { rnd.nextDouble }
+ val b = Array.fill(k * n) { rnd.nextDouble() }
val ldb = k
- val beta = rnd.nextDouble
- val c = Array.fill(m * n) { rnd.nextDouble }
+ val beta = rnd.nextDouble()
+ val c = Array.fill(m * n) { rnd.nextDouble() }
val ldc = m
runBLASBenchmark("dgemm[T,N]", m * n * k) { impl =>
@@ -357,13 +357,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextDouble
- val a = Array.fill(m * k) { rnd.nextDouble }
+ val alpha = rnd.nextDouble()
+ val a = Array.fill(m * k) { rnd.nextDouble() }
val lda = k
- val b = Array.fill(k * n) { rnd.nextDouble }
+ val b = Array.fill(k * n) { rnd.nextDouble() }
val ldb = n
- val beta = rnd.nextDouble
- val c = Array.fill(m * n) { rnd.nextDouble }
+ val beta = rnd.nextDouble()
+ val c = Array.fill(m * n) { rnd.nextDouble() }
val ldc = m
runBLASBenchmark("dgemm[T,T]", m * n * k) { impl =>
@@ -375,13 +375,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextFloat
- val a = Array.fill(m * k) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val a = Array.fill(m * k) { rnd.nextFloat() }
val lda = m
- val b = Array.fill(k * n) { rnd.nextFloat }
+ val b = Array.fill(k * n) { rnd.nextFloat() }
val ldb = k
- val beta = rnd.nextFloat
- val c = Array.fill(m * n) { rnd.nextFloat }
+ val beta = rnd.nextFloat()
+ val c = Array.fill(m * n) { rnd.nextFloat() }
val ldc = m
runBLASBenchmark("sgemm[N,N]", m * n * k) { impl =>
@@ -393,13 +393,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextFloat
- val a = Array.fill(m * k) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val a = Array.fill(m * k) { rnd.nextFloat() }
val lda = m
- val b = Array.fill(k * n) { rnd.nextFloat }
+ val b = Array.fill(k * n) { rnd.nextFloat() }
val ldb = n
- val beta = rnd.nextFloat
- val c = Array.fill(m * n) { rnd.nextFloat }
+ val beta = rnd.nextFloat()
+ val c = Array.fill(m * n) { rnd.nextFloat() }
val ldc = m
runBLASBenchmark("sgemm[N,T]", m * n * k) { impl =>
@@ -411,13 +411,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextFloat
- val a = Array.fill(m * k) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val a = Array.fill(m * k) { rnd.nextFloat() }
val lda = k
- val b = Array.fill(k * n) { rnd.nextFloat }
+ val b = Array.fill(k * n) { rnd.nextFloat() }
val ldb = k
- val beta = rnd.nextFloat
- val c = Array.fill(m * n) { rnd.nextFloat }
+ val beta = rnd.nextFloat()
+ val c = Array.fill(m * n) { rnd.nextFloat() }
val ldc = m
runBLASBenchmark("sgemm[T,N]", m * n * k) { impl =>
@@ -429,13 +429,13 @@
val m = 1e3.toInt
val n = 1e3.toInt
val k = 1e3.toInt
- val alpha = rnd.nextFloat
- val a = Array.fill(m * k) { rnd.nextFloat }
+ val alpha = rnd.nextFloat()
+ val a = Array.fill(m * k) { rnd.nextFloat() }
val lda = k
- val b = Array.fill(k * n) { rnd.nextFloat }
+ val b = Array.fill(k * n) { rnd.nextFloat() }
val ldb = n
- val beta = rnd.nextFloat
- val c = Array.fill(m * n) { rnd.nextFloat }
+ val beta = rnd.nextFloat()
+ val c = Array.fill(m * n) { rnd.nextFloat() }
val ldc = m
runBLASBenchmark("sgemm[T,T]", m * n * k) { impl =>
diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala
index f086697..f991f35 100644
--- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala
+++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala
@@ -234,11 +234,11 @@
val nnz = random.nextInt(m)
val indices1 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
- val values1 = Array.fill(nnz)(random.nextDouble)
+ val values1 = Array.fill(nnz)(random.nextDouble())
val sparseVector1 = Vectors.sparse(m, indices1, values1)
val indices2 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
- val values2 = Array.fill(nnz)(random.nextDouble)
+ val values2 = Array.fill(nnz)(random.nextDouble())
val sparseVector2 = Vectors.sparse(m, indices2, values2)
val denseVector1 = Vectors.dense(sparseVector1.toArray)
@@ -454,8 +454,8 @@
valuesBuilder += v
}
val (indices, values) = vec.activeIterator.toArray.unzip
- assert(indicesBuilder.result === indices)
- assert(valuesBuilder.result === values)
+ assert(indicesBuilder.result() === indices)
+ assert(valuesBuilder.result() === values)
}
}
@@ -475,8 +475,8 @@
}
}
val (indices, values) = vec.nonZeroIterator.toArray.unzip
- assert(indicesBuilder.result === indices)
- assert(valuesBuilder.result === values)
+ assert(indicesBuilder.result() === indices)
+ assert(valuesBuilder.result() === values)
}
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
index f35c6e0..b342027 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
@@ -307,7 +307,7 @@
override def transform(dataset: Dataset[_]): DataFrame = instrumented(instr =>
instr.withTransformEvent(this, dataset) {
transformSchema(dataset.schema, logging = true)
- stages.foldLeft(dataset.toDF)((cur, transformer) =>
+ stages.foldLeft(dataset.toDF())((cur, transformer) =>
instr.withTransformEvent(transformer, cur)(transformer.transform(cur)))
})
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala
index 9c6eb88..41f3946 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala
@@ -194,7 +194,7 @@
} else {
this.logWarning(s"$uid: Predictor.transform() does nothing" +
" because no output columns were set.")
- dataset.toDF
+ dataset.toDF()
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala
index 91dd6ab..992e91e 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala
@@ -179,7 +179,7 @@
sum
}
- override def toString: String = toMetadata.toString
+ override def toString: String = toMetadata().toString
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala
index c46be17..e12c68f 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala
@@ -152,7 +152,7 @@
logWarning(s"$uid: ClassificationModel.transform() does nothing" +
" because no output columns were set.")
}
- outputData.toDF
+ outputData.toDF()
}
final override def transformImpl(dataset: Dataset[_]): DataFrame =
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala
index 54f3c1e..13898a3 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala
@@ -342,7 +342,7 @@
val adapt = BLAS.javaBLAS.ddot(numFeatures, solution, 1, scaledMean, 1)
solution(numFeatures) -= adapt
}
- (solution, arrayBuilder.result)
+ (solution, arrayBuilder.result())
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
index adf77eb..0efa57e 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
@@ -1036,7 +1036,7 @@
solution(numFeatures) -= adapt
}
}
- (solution, arrayBuilder.result)
+ (solution, arrayBuilder.result())
}
@Since("1.4.0")
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala
index cf94c9f..52106f4 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala
@@ -182,7 +182,7 @@
if (getPredictionCol.isEmpty && getRawPredictionCol.isEmpty) {
logWarning(s"$uid: OneVsRestModel.transform() does nothing" +
" because no output columns were set.")
- return dataset.toDF
+ return dataset.toDF()
}
val isProbModel = models.head.isInstanceOf[ProbabilisticClassificationModel[_, _]]
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
index 1caaecc..460f239 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
@@ -157,7 +157,7 @@
this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() does nothing" +
" because no output columns were set.")
}
- outputData.toDF
+ outputData.toDF()
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala
index 0331555..98ab5fa 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala
@@ -145,7 +145,7 @@
this.logWarning(s"$uid: GaussianMixtureModel.transform() does nothing" +
" because no output columns were set.")
}
- outputData.toDF
+ outputData.toDF()
}
@Since("2.0.0")
@@ -441,7 +441,7 @@
instances.mapPartitions { iter =>
if (iter.nonEmpty) {
val agg = new ExpectationAggregator(numFeatures, bcWeights, bcGaussians)
- while (iter.hasNext) { agg.add(iter.next) }
+ while (iter.hasNext) { agg.add(iter.next()) }
// sum of weights in this partition
val ws = agg.weights.sum
if (iteration == 0) weightSumAccum.add(ws)
diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala
index d4c8781..8b2ee95 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala
@@ -178,7 +178,7 @@
.setMaxIterations($(maxIter))
val model = algorithm.run(rdd)
- model.assignments.toDF
+ model.assignments.toDF()
}
@Since("2.4.0")
diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala
index 143e26f..85242e4 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala
@@ -106,7 +106,7 @@
val metrics = getMetrics(dataset)
$(metricName) match {
- case ("silhouette") => metrics.silhouette
+ case ("silhouette") => metrics.silhouette()
case (other) =>
throw new IllegalArgumentException(s"No support for metric $other")
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala
index 16d7111..e073e41 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala
@@ -179,7 +179,7 @@
inputDim: Int): BucketedRandomProjectionLSHModel = {
val rng = new Random($(seed))
val localNumHashTables = $(numHashTables)
- val values = Array.fill(localNumHashTables * inputDim)(rng.nextGaussian)
+ val values = Array.fill(localNumHashTables * inputDim)(rng.nextGaussian())
var i = 0
while (i < localNumHashTables) {
val offset = i * inputDim
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala
index ca03409..74be13a 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala
@@ -194,7 +194,7 @@
if (dataset.storageLevel == StorageLevel.NONE) {
input.persist(StorageLevel.MEMORY_AND_DISK)
}
- Some(input.count)
+ Some(input.count())
} else {
None
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala
index 5254762..2515365 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala
@@ -142,13 +142,13 @@
var s = new QuantileSummaries(
QuantileSummaries.defaultCompressThreshold, relativeError)
while (iter.hasNext) {
- val row = iter.next
+ val row = iter.next()
if (!row.isNullAt(0)) {
val v = row.getDouble(0)
if (!v.isNaN) s = s.insert(v)
}
}
- Iterator.single(s.compress)
+ Iterator.single(s.compress())
} else Iterator.empty
}.treeReduce((s1, s2) => s1.merge(s2))
val count = summary.count
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala
index d189edc..cdedcc2 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala
@@ -79,8 +79,8 @@
return 1.0
}
- var xIndex = xIter.next
- var yIndex = yIter.next
+ var xIndex = xIter.next()
+ var yIndex = yIter.next()
var xSize = 1
var ySize = 1
var intersectionSize = 0
@@ -88,12 +88,12 @@
while (xIndex != -1 && yIndex != -1) {
if (xIndex == yIndex) {
intersectionSize += 1
- xIndex = if (xIter.hasNext) { xSize += 1; xIter.next } else -1
- yIndex = if (yIter.hasNext) { ySize += 1; yIter.next } else -1
+ xIndex = if (xIter.hasNext) { xSize += 1; xIter.next() } else -1
+ yIndex = if (yIter.hasNext) { ySize += 1; yIter.next() } else -1
} else if (xIndex > yIndex) {
- yIndex = if (yIter.hasNext) { ySize += 1; yIter.next } else -1
+ yIndex = if (yIter.hasNext) { ySize += 1; yIter.next() } else -1
} else {
- xIndex = if (xIter.hasNext) { xSize += 1; xIter.next } else -1
+ xIndex = if (xIter.hasNext) { xSize += 1; xIter.next() } else -1
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala
index 08fe750..9387ab3 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala
@@ -386,7 +386,7 @@
private def transformLabel(dataset: Dataset[_]): DataFrame = {
val labelName = resolvedFormula.label
if (labelName.isEmpty || hasLabelCol(dataset.schema)) {
- dataset.toDF
+ dataset.toDF()
} else if (dataset.schema.exists(_.name == labelName)) {
dataset.schema(labelName).dataType match {
case _: NumericType | BooleanType =>
@@ -397,7 +397,7 @@
} else {
// Ignore the label field. This is a hack so that this transformer can also work on test
// datasets in a Pipeline.
- dataset.toDF
+ dataset.toDF()
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala
index 85352d6..df6e54c 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala
@@ -190,10 +190,10 @@
val summaries = Array.fill(numFeatures)(
new QuantileSummaries(QuantileSummaries.defaultCompressThreshold, relativeError))
while (iter.hasNext) {
- val vec = iter.next
+ val vec = iter.next()
vec.foreach { (i, v) => if (!v.isNaN) summaries(i) = summaries(i).insert(v) }
}
- Iterator.tabulate(numFeatures)(i => (i, summaries(i).compress))
+ Iterator.tabulate(numFeatures)(i => (i, summaries(i).compress()))
} else Iterator.empty
}.reduceByKey { (s1, s2) => s1.merge(s2) }
} else {
@@ -206,9 +206,9 @@
}.aggregateByKey(
new QuantileSummaries(QuantileSummaries.defaultCompressThreshold, relativeError))(
seqOp = (s, v) => s.insert(v),
- combOp = (s1, s2) => s1.compress.merge(s2.compress)
+ combOp = (s1, s2) => s1.compress().merge(s2.compress())
).map { case ((_, i), s) => (i, s)
- }.reduceByKey { (s1, s2) => s1.compress.merge(s2.compress) }
+ }.reduceByKey { (s1, s2) => s1.compress().merge(s2.compress()) }
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala
index 1afab32..8ff880b 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala
@@ -207,7 +207,7 @@
import spark.implicits._
val numFeatures = DatasetUtils.getNumFeatures(dataset, $(featuresCol))
- val resultDF = getSelectionTestResult(dataset.toDF)
+ val resultDF = getSelectionTestResult(dataset.toDF())
def getTopIndices(k: Int): Array[Int] = {
resultDF.sort("pValue", "featureIndex")
@@ -233,7 +233,7 @@
val maxIndex = resultDF.sort("pValue", "featureIndex")
.select("pValue")
.as[Double].rdd
- .zipWithIndex
+ .zipWithIndex()
.flatMap { case (pValue, index) =>
if (pValue <= f * (index + 1)) {
Iterator.single(index.toInt)
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala
index 4f11c58..7f8850b 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala
@@ -199,7 +199,7 @@
val selectedCols = getSelectedCols(dataset, inputCols)
dataset.select(selectedCols: _*)
- .toDF
+ .toDF()
.agg(aggregator.toColumn)
.as[Array[OpenHashMap[String, Long]]]
.collect()(0)
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala
index 3b43404..6142447 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala
@@ -179,11 +179,11 @@
val resultDF = ($(featureType), $(labelType)) match {
case ("categorical", "categorical") =>
- ChiSquareTest.test(dataset.toDF, getFeaturesCol, getLabelCol, true)
+ ChiSquareTest.test(dataset.toDF(), getFeaturesCol, getLabelCol, true)
case ("continuous", "categorical") =>
- ANOVATest.test(dataset.toDF, getFeaturesCol, getLabelCol, true)
+ ANOVATest.test(dataset.toDF(), getFeaturesCol, getLabelCol, true)
case ("continuous", "continuous") =>
- FValueTest.test(dataset.toDF, getFeaturesCol, getLabelCol, true)
+ FValueTest.test(dataset.toDF(), getFeaturesCol, getLabelCol, true)
case _ =>
throw new IllegalArgumentException(s"Unsupported combination:" +
s" featureType=${$(featureType)}, labelType=${$(labelType)}")
@@ -227,7 +227,7 @@
val maxIndex = resultDF.sort("pValue", "featureIndex")
.select("pValue")
.as[Double].rdd
- .zipWithIndex
+ .zipWithIndex()
.flatMap { case (pValue, index) =>
if (pValue <= f * (index + 1)) {
Iterator.single(index.toInt)
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala
index fd44b1c..5687ba8 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala
@@ -153,7 +153,7 @@
}
val numFeaturesSelected = $(indices).length + $(names).length
val outputAttr = new AttributeGroup($(outputCol), numFeaturesSelected)
- SchemaUtils.appendColumn(schema, outputAttr.toStructField)
+ SchemaUtils.appendColumn(schema, outputAttr.toStructField())
}
@Since("1.5.0")
diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala
index 3b988fb..e9abcb0 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala
@@ -343,7 +343,7 @@
val adapt = BLAS.getBLAS(numFeatures).ddot(numFeatures, solution, 1, scaledMean, 1)
solution(numFeatures) -= adapt
}
- (solution, arrayBuilder.result)
+ (solution, arrayBuilder.result())
}
@Since("1.6.0")
diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala
index 1520220..6e26a78 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala
@@ -1077,7 +1077,7 @@
this.logWarning(s"$uid: GeneralizedLinearRegressionModel.transform() does nothing" +
" because no output columns were set.")
}
- outputData.toDF
+ outputData.toDF()
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala
index 7295ce6..9638eee 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala
@@ -612,7 +612,7 @@
val adapt = BLAS.javaBLAS.ddot(numFeatures, solution, 1, scaledMean, 1)
solution(numFeatures) -= adapt
}
- (solution, arrayBuilder.result)
+ (solution, arrayBuilder.result())
}
private def createModel(
diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala
index 800c68d..89579df 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala
@@ -124,7 +124,7 @@
if (iter.hasNext) {
val array = Array.ofDim[Double](numFeatures)
while (iter.hasNext) {
- val (label, features) = iter.next
+ val (label, features) = iter.next()
val yDiff = label - yMean
if (yDiff != 0) {
features.iterator.zip(xMeans.iterator)
diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala
index 19ea7e4..8cf19f2 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala
@@ -317,7 +317,7 @@
// Prepare periodic checkpointers
// Note: this is checkpointing the unweighted training error
val predErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)](
- treeStrategy.getCheckpointInterval, sc, StorageLevel.MEMORY_AND_DISK)
+ treeStrategy.getCheckpointInterval(), sc, StorageLevel.MEMORY_AND_DISK)
timer.stop("init")
@@ -393,7 +393,7 @@
validatePredError = computeInitialPredictionAndError(
validationTreePoints, firstTreeWeight, firstTreeModel, loss, bcSplits)
validatePredErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)](
- treeStrategy.getCheckpointInterval, sc, StorageLevel.MEMORY_AND_DISK)
+ treeStrategy.getCheckpointInterval(), sc, StorageLevel.MEMORY_AND_DISK)
validatePredErrorCheckpointer.update(validatePredError)
bestValidateError = computeWeightedError(validationTreePoints, validatePredError)
timer.stop("init validation")
diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala
index de6c935..012e942 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala
@@ -163,7 +163,7 @@
// At first, all the rows belong to the root nodes (node Id == 1).
nodeIds = baggedInput.map { _ => Array.fill(numTrees)(1) }
nodeIdCheckpointer = new PeriodicRDDCheckpointer[Array[Int]](
- strategy.getCheckpointInterval, sc, StorageLevel.MEMORY_AND_DISK)
+ strategy.getCheckpointInterval(), sc, StorageLevel.MEMORY_AND_DISK)
nodeIdCheckpointer.update(nodeIds)
}
@@ -232,7 +232,7 @@
if (strategy.algo == OldAlgo.Classification) {
topNodes.map { rootNode =>
new DecisionTreeClassificationModel(uid, rootNode.toNode(prune), numFeatures,
- strategy.getNumClasses)
+ strategy.getNumClasses())
}
} else {
topNodes.map { rootNode =>
@@ -243,7 +243,7 @@
if (strategy.algo == OldAlgo.Classification) {
topNodes.map { rootNode =>
new DecisionTreeClassificationModel(rootNode.toNode(prune), numFeatures,
- strategy.getNumClasses)
+ strategy.getNumClasses())
}
} else {
topNodes.map(rootNode =>
diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
index 4e9dadd..f186b24 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
@@ -158,10 +158,10 @@
// Compute metrics for each model over each split
val (splits, schemaWithoutFold) = if ($(foldCol) == "") {
- (MLUtils.kFold(dataset.toDF.rdd, $(numFolds), $(seed)), schema)
+ (MLUtils.kFold(dataset.toDF().rdd, $(numFolds), $(seed)), schema)
} else {
val filteredSchema = StructType(schema.filter(_.name != $(foldCol)).toArray)
- (MLUtils.kFold(dataset.toDF, $(numFolds), $(foldCol)), filteredSchema)
+ (MLUtils.kFold(dataset.toDF(), $(numFolds), $(foldCol)), filteredSchema)
}
val metrics = splits.zipWithIndex.map { case ((training, validation), splitIndex) =>
val trainingDataset = sparkSession.createDataFrame(training, schemaWithoutFold).cache()
diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala
index 130790a..08ecdaf 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala
@@ -215,7 +215,7 @@
*/
private[ml] def getNumFeatures(dataset: Dataset[_], vectorCol: String): Int = {
MetadataUtils.getNumFeatures(dataset.schema(vectorCol)).getOrElse {
- dataset.select(columnToVector(dataset, vectorCol)).head.getAs[Vector](0).size
+ dataset.select(columnToVector(dataset, vectorCol)).head().getAs[Vector](0).size
}
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala
index c08d7e8..e3f93b3 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala
@@ -120,7 +120,7 @@
size: Int): StructType = {
require(size > 0)
val attrGroup = new AttributeGroup(colName, size)
- val field = attrGroup.toStructField
+ val field = attrGroup.toStructField()
updateField(schema, field, true)
}
@@ -138,7 +138,7 @@
val attr = NominalAttribute.defaultAttr
.withName(colName)
.withNumValues(numValues)
- val field = attr.toStructField
+ val field = attr.toStructField()
updateField(schema, field, true)
}
@@ -153,7 +153,7 @@
colName: String): StructType = {
val attr = NumericAttribute.defaultAttr
.withName(colName)
- val field = attr.toStructField
+ val field = attr.toStructField()
updateField(schema, field, true)
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala
index 6be32ab..a53581e 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala
@@ -157,7 +157,7 @@
instances: RDD[(Vector, Double)],
handlePersistence: Boolean,
instr: Option[Instrumentation]): BisectingKMeansModel = {
- val d = instances.map(_._1.size).first
+ val d = instances.map(_._1.size).first()
logInfo(s"Feature dimension: $d.")
val dMeasure = DistanceMeasure.decodeFromString(this.distanceMeasure)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala
index e4c29a7..5b0fb5e 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala
@@ -101,7 +101,7 @@
} else Iterator.empty
}
}
- }.collect.foreach { case (i, j, s) =>
+ }.collect().foreach { case (i, j, s) =>
val index = indexUpperTriangular(k, i, j)
packedValues(index) = s
if (s < diagValues(i)) diagValues(i) = s
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala
index 32471b0..ea548b2 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala
@@ -39,7 +39,7 @@
*/
private def toPMML(streamResult: StreamResult): Unit = {
val pmmlModelExport = PMMLModelExportFactory.createPMMLModelExport(this)
- JAXBUtil.marshalPMML(pmmlModelExport.getPmml, streamResult)
+ JAXBUtil.marshalPMML(pmmlModelExport.getPmml(), streamResult)
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala
index eb83f9a..94848cb 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala
@@ -88,7 +88,7 @@
throw new IllegalArgumentException("Model must be initialized before starting training.")
}
data.foreachRDD { (rdd, time) =>
- if (!rdd.isEmpty) {
+ if (!rdd.isEmpty()) {
model = Some(algorithm.run(rdd, model.get.weights))
logInfo(s"Model updated at time ${time.toString}")
val display = model.get.weights.size match {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index de21db8..f6f5edb 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
@@ -95,7 +95,7 @@
// optionally add gaussian noise
if (noise) {
- trainData.map(x => (x._1, x._2, x._3 + rand.nextGaussian * sigma))
+ trainData.map(x => (x._1, x._2, x._3 + rand.nextGaussian() * sigma))
}
trainData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath)
diff --git a/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala
index 1f10808..d166b4b 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala
@@ -52,7 +52,7 @@
for ((colName, valType) <- Seq(
("vec", "null"), ("oldVec", "null"), ("label", "java.lang.Integer"))) {
val thrown1 = intercept[SparkException] {
- df2.select(vector_to_array(col(colName))).count
+ df2.select(vector_to_array(col(colName))).count()
}
assert(thrown1.getCause.getMessage.contains(
"function vector_to_array requires a non-null input argument and input type must be " +
diff --git a/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala
index d58c938..e28b260 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala
@@ -88,11 +88,11 @@
val dataset4 = mock[DataFrame]
val dataset5 = mock[DataFrame]
- when(dataset1.toDF).thenReturn(dataset1)
- when(dataset2.toDF).thenReturn(dataset2)
- when(dataset3.toDF).thenReturn(dataset3)
- when(dataset4.toDF).thenReturn(dataset4)
- when(dataset5.toDF).thenReturn(dataset5)
+ when(dataset1.toDF()).thenReturn(dataset1)
+ when(dataset2.toDF()).thenReturn(dataset2)
+ when(dataset3.toDF()).thenReturn(dataset3)
+ when(dataset4.toDF()).thenReturn(dataset4)
+ when(dataset5.toDF()).thenReturn(dataset5)
when(estimator1.fit(meq(dataset1))).thenReturn(model1)
when(model1.transform(meq(dataset1))).thenReturn(dataset2)
@@ -153,10 +153,10 @@
val dataset2 = mock[DataFrame]
val dataset3 = mock[DataFrame]
val dataset4 = mock[DataFrame]
- when(dataset1.toDF).thenReturn(dataset1)
- when(dataset2.toDF).thenReturn(dataset2)
- when(dataset3.toDF).thenReturn(dataset3)
- when(dataset4.toDF).thenReturn(dataset4)
+ when(dataset1.toDF()).thenReturn(dataset1)
+ when(dataset2.toDF()).thenReturn(dataset2)
+ when(dataset3.toDF()).thenReturn(dataset3)
+ when(dataset4.toDF()).thenReturn(dataset4)
val transformer1 = mock[Transformer]
val model = mock[MyModel]
diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
index e9c08f0..3bf339f 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
@@ -53,11 +53,11 @@
val dataset3 = mock[DataFrame]
val dataset4 = mock[DataFrame]
- when(dataset0.toDF).thenReturn(dataset0)
- when(dataset1.toDF).thenReturn(dataset1)
- when(dataset2.toDF).thenReturn(dataset2)
- when(dataset3.toDF).thenReturn(dataset3)
- when(dataset4.toDF).thenReturn(dataset4)
+ when(dataset0.toDF()).thenReturn(dataset0)
+ when(dataset1.toDF()).thenReturn(dataset1)
+ when(dataset2.toDF()).thenReturn(dataset2)
+ when(dataset3.toDF()).thenReturn(dataset3)
+ when(dataset4.toDF()).thenReturn(dataset4)
when(estimator0.copy(any[ParamMap])).thenReturn(estimator0)
when(model0.copy(any[ParamMap])).thenReturn(model0)
@@ -247,7 +247,7 @@
override def write: MLWriter = new DefaultParamsWriter(this)
- override def transform(dataset: Dataset[_]): DataFrame = dataset.toDF
+ override def transform(dataset: Dataset[_]): DataFrame = dataset.toDF()
override def transformSchema(schema: StructType): StructType = schema
}
@@ -270,7 +270,7 @@
override def copy(extra: ParamMap): UnWritableStage = defaultCopy(extra)
- override def transform(dataset: Dataset[_]): DataFrame = dataset.toDF
+ override def transform(dataset: Dataset[_]): DataFrame = dataset.toDF()
override def transformSchema(schema: StructType): StructType = schema
}
diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala
index 8a4fea2..0994465 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala
@@ -369,13 +369,13 @@
.setMaxDepth(2)
.setMaxIter(3)
.setLossType("logistic")
- val model3 = gbt.fit(trainData.toDF)
+ val model3 = gbt.fit(trainData.toDF())
val model1 = new GBTClassificationModel("gbt-cls-model-test1",
model3.trees.take(1), model3.treeWeights.take(1), model3.numFeatures, model3.numClasses)
val model2 = new GBTClassificationModel("gbt-cls-model-test2",
model3.trees.take(2), model3.treeWeights.take(2), model3.numFeatures, model3.numClasses)
- val evalArr = model3.evaluateEachIteration(validationData.toDF)
+ val evalArr = model3.evaluateEachIteration(validationData.toDF())
val remappedValidationData = validationData.map {
case LabeledPoint(label, features) =>
Instance(label * 2 - 1, 1.0, features)
diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
index afc57a3..50d8bcb 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
@@ -2587,7 +2587,7 @@
blorModel.evaluate(smallBinaryDataset).asInstanceOf[BinaryLogisticRegressionSummary]
assert(blorSummary.areaUnderROC === sameBlorSummary.areaUnderROC)
assert(blorSummary.roc.collect() === sameBlorSummary.roc.collect())
- assert(blorSummary.pr.collect === sameBlorSummary.pr.collect())
+ assert(blorSummary.pr.collect() === sameBlorSummary.pr.collect())
assert(
blorSummary.fMeasureByThreshold.collect() === sameBlorSummary.fMeasureByThreshold.collect())
assert(
@@ -3143,7 +3143,7 @@
for (i <- 0 until nClasses) {
if (p < probs(i)) {
y = i
- break
+ break()
}
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
index baeebfb..0077e03 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
@@ -152,13 +152,13 @@
.setPredictionCol("label")
val metrics1 = evaluator.getMetrics(irisDataset)
- val silhouetteScoreEuclidean = metrics1.silhouette
+ val silhouetteScoreEuclidean = metrics1.silhouette()
assert(evaluator.evaluate(irisDataset) == silhouetteScoreEuclidean)
evaluator.setDistanceMeasure("cosine")
val metrics2 = evaluator.getMetrics(irisDataset)
- val silhouetteScoreCosin = metrics2.silhouette
+ val silhouetteScoreCosin = metrics2.silhouette()
assert(evaluator.evaluate(irisDataset) == silhouetteScoreCosin)
}
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala
index 8f8365a..bac605c 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala
@@ -34,14 +34,14 @@
import testImplicits._
test("forward transform of discrete cosine matches jTransforms result") {
- val data = Vectors.dense((0 until 128).map(_ => 2D * math.random - 1D).toArray)
+ val data = Vectors.dense((0 until 128).map(_ => 2D * math.random() - 1D).toArray)
val inverse = false
testDCT(data, inverse)
}
test("inverse transform of discrete cosine matches jTransforms result") {
- val data = Vectors.dense((0 until 128).map(_ => 2D * math.random - 1D).toArray)
+ val data = Vectors.dense((0 until 128).map(_ => 2D * math.random() - 1D).toArray)
val inverse = true
testDCT(data, inverse)
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala
index 682b87a..542eb17 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala
@@ -72,7 +72,7 @@
val model = discretizer.fit(df)
testTransformerByGlobalCheckFunc[(Double)](df, model, "result") { rows =>
val result = rows.map { r => Tuple1(r.getDouble(0)) }.toDF("result")
- val observedNumBuckets = result.select("result").distinct.count
+ val observedNumBuckets = result.select("result").distinct().count()
assert(observedNumBuckets == expectedNumBuckets,
s"Observed number of buckets are not correct." +
s" Expected $expectedNumBuckets but found $observedNumBuckets")
@@ -130,8 +130,8 @@
val model = discretizer.fit(trainDF)
testTransformerByGlobalCheckFunc[(Double)](testDF, model, "result") { rows =>
val result = rows.map { r => Tuple1(r.getDouble(0)) }.toDF("result")
- val firstBucketSize = result.filter(result("result") === 0.0).count
- val lastBucketSize = result.filter(result("result") === 4.0).count
+ val firstBucketSize = result.filter(result("result") === 0.0).count()
+ val lastBucketSize = result.filter(result("result") === 4.0).count()
assert(firstBucketSize === 30L,
s"Size of first bucket ${firstBucketSize} did not equal expected value of 30.")
@@ -221,7 +221,7 @@
val result =
rows.map { r => Tuple2(r.getDouble(0), r.getDouble(1)) }.toDF("result1", "result2")
for (i <- 1 to 2) {
- val observedNumBuckets = result.select("result" + i).distinct.count
+ val observedNumBuckets = result.select("result" + i).distinct().count()
assert(observedNumBuckets == expectedNumBucket,
s"Observed number of buckets are not correct." +
s" Expected $expectedNumBucket but found ($observedNumBuckets")
@@ -508,7 +508,7 @@
val model = discretizer.fit(df)
val result = model.transform(df)
- val observedNumBuckets = result.select(discretizer.getOutputCol).distinct.count
+ val observedNumBuckets = result.select(discretizer.getOutputCol).distinct().count()
assert(observedNumBuckets === numBuckets,
"Observed number of buckets does not equal expected number of buckets.")
}
@@ -517,7 +517,7 @@
import scala.util.Random
val rng = new Random(3)
- val a1 = Array.tabulate(200)(_ => rng.nextDouble * 2.0 - 1.0) ++
+ val a1 = Array.tabulate(200)(_ => rng.nextDouble() * 2.0 - 1.0) ++
Array.fill(20)(0.0) ++ Array.fill(20)(-0.0)
val df1 = sc.parallelize(a1, 2).toDF("id")
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/UnivariateFeatureSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/UnivariateFeatureSelectorSuite.scala
index 84868dc..e83d0f6 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/UnivariateFeatureSelectorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/UnivariateFeatureSelectorSuite.scala
@@ -506,8 +506,8 @@
val dataset_f_classification2 =
spark.createDataFrame(data_f_classif2).toDF("label", "features", "topFeature")
- val resultDF1 = ANOVATest.test(dataset_f_classification1.toDF, "features", "label", true)
- val resultDF2 = ANOVATest.test(dataset_f_classification2.toDF, "features", "label", true)
+ val resultDF1 = ANOVATest.test(dataset_f_classification1.toDF(), "features", "label", true)
+ val resultDF2 = ANOVATest.test(dataset_f_classification2.toDF(), "features", "label", true)
val selector = new UnivariateFeatureSelector()
.setOutputCol("filtered")
.setFeatureType("continuous")
@@ -632,8 +632,8 @@
val dataset_f_regression2 =
spark.createDataFrame(data_f_regression2).toDF("label", "features", "topFeature")
- val resultDF1 = FValueTest.test(dataset_f_regression1.toDF, "features", "label", true)
- val resultDF2 = FValueTest.test(dataset_f_regression2.toDF, "features", "label", true)
+ val resultDF1 = FValueTest.test(dataset_f_regression1.toDF(), "features", "label", true)
+ val resultDF2 = FValueTest.test(dataset_f_regression2.toDF(), "features", "label", true)
val selector = new UnivariateFeatureSelector()
.setOutputCol("filtered")
.setFeatureType("continuous")
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala
index d89d10b..14faa81 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala
@@ -198,7 +198,7 @@
val b = Vectors.sparse(4, Array(0, 3), Array(3, 6))
val stream = MemoryStream[(Vector, Vector)]
- val streamingDF = stream.toDS.toDF("a", "b")
+ val streamingDF = stream.toDS().toDF("a", "b")
val sizeHintA = new VectorSizeHint()
.setSize(3)
.setInputCol("a")
diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala
index 6d14350..2ae5498 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala
@@ -116,7 +116,7 @@
var idx = 0
for (fitIntercept <- Seq(false, true)) {
- val yMean = instances2.map(_.label).mean
+ val yMean = instances2.map(_.label).mean()
val newInstances = instances2.map { instance =>
val mu = (instance.label + yMean) / 2.0
val eta = math.log(mu)
diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala
index f5ae22d..9ece1da 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/BinaryLogisticBlockAggregatorSuite.scala
@@ -113,8 +113,8 @@
test("check sizes") {
val rng = new scala.util.Random
val numFeatures = instances.head.features.size
- val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble))
- val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble))
+ val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble()))
+ val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble()))
val block = InstanceBlock.fromInstances(instances)
val aggIntercept = getNewAggregator(instances, coefWithIntercept,
diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeBlockAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeBlockAggregatorSuite.scala
index 029911a..f0167a6 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeBlockAggregatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeBlockAggregatorSuite.scala
@@ -112,8 +112,8 @@
test("check sizes") {
val rng = new scala.util.Random
val numFeatures = instances.head.features.size
- val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble))
- val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble))
+ val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble()))
+ val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble()))
val block = InstanceBlock.fromInstances(instances)
val aggIntercept = getNewAggregator(instances, coefWithIntercept, fitIntercept = true)
diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberBlockAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberBlockAggregatorSuite.scala
index df1a1d3..4bad50f 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberBlockAggregatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberBlockAggregatorSuite.scala
@@ -114,8 +114,8 @@
test("check sizes") {
val rng = new scala.util.Random
val numFeatures = instances.head.features.size
- val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble))
- val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble))
+ val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble()))
+ val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble()))
val block = InstanceBlock.fromInstances(instances)
val aggIntercept = getNewAggregator(instances, coefWithIntercept,
diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresBlockAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresBlockAggregatorSuite.scala
index b71951c..11020eda 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresBlockAggregatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresBlockAggregatorSuite.scala
@@ -110,7 +110,7 @@
test("check sizes") {
val rng = new scala.util.Random
val numFeatures = instances.head.features.size
- val coefVec = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble))
+ val coefVec = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble()))
val block = InstanceBlock.fromInstances(instances)
val agg = getNewAggregator(instances, coefVec, fitIntercept = true)
diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala
index d00fdac..8200085 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/MultinomialLogisticBlockAggregatorSuite.scala
@@ -115,9 +115,9 @@
val numFeatures = instances.head.features.size
val numClasses = instances.map(_.label).distinct.size
val coefWithIntercept = Vectors.dense(
- Array.fill(numClasses * (numFeatures + 1))(rng.nextDouble))
+ Array.fill(numClasses * (numFeatures + 1))(rng.nextDouble()))
val coefWithoutIntercept = Vectors.dense(
- Array.fill(numClasses * numFeatures)(rng.nextDouble))
+ Array.fill(numClasses * numFeatures)(rng.nextDouble()))
val block = InstanceBlock.fromInstances(instances)
val aggIntercept = getNewAggregator(instances, coefWithIntercept,
diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
index 7ad26c0..d206b5f 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
@@ -778,7 +778,7 @@
import org.apache.spark.sql.functions._
val (ratings, _) = genExplicitTestData(numUsers = 4, numItems = 4, rank = 1)
- val data = ratings.toDF
+ val data = ratings.toDF()
val knownUser = data.select(max("user")).as[Int].first()
val unknownUser = knownUser + 10
val knownItem = data.select(max("item")).as[Int].first()
@@ -815,7 +815,7 @@
val spark = this.spark
import spark.implicits._
val (ratings, _) = genExplicitTestData(numUsers = 2, numItems = 2, rank = 1)
- val data = ratings.toDF
+ val data = ratings.toDF()
val model = new ALS().fit(data)
Seq("nan", "NaN", "Nan", "drop", "DROP", "Drop").foreach { s =>
testTransformer[Rating[Int]](data, model.setColdStartStrategy(s), "prediction") { _ => }
@@ -845,8 +845,8 @@
test("recommendForAllUsers with k <, = and > num_items") {
val model = getALSModel
- val numUsers = model.userFactors.count
- val numItems = model.itemFactors.count
+ val numUsers = model.userFactors.count()
+ val numItems = model.itemFactors.count()
val expected = Map(
0 -> Seq((3, 54f), (4, 44f), (5, 42f), (6, 28f)),
1 -> Seq((3, 39f), (5, 33f), (4, 26f), (6, 16f)),
@@ -865,8 +865,8 @@
test("recommendForAllItems with k <, = and > num_users") {
val model = getALSModel
- val numUsers = model.userFactors.count
- val numItems = model.itemFactors.count
+ val numUsers = model.userFactors.count()
+ val numItems = model.itemFactors.count()
val expected = Map(
3 -> Seq((0, 54f), (2, 51f), (1, 39f)),
4 -> Seq((0, 44f), (2, 30f), (1, 26f)),
@@ -888,13 +888,13 @@
val spark = this.spark
import spark.implicits._
val model = getALSModel
- val numItems = model.itemFactors.count
+ val numItems = model.itemFactors.count()
val expected = Map(
0 -> Seq((3, 54f), (4, 44f), (5, 42f), (6, 28f)),
2 -> Seq((3, 51f), (5, 45f), (4, 30f), (6, 18f))
)
val userSubset = expected.keys.toSeq.toDF("user")
- val numUsersSubset = userSubset.count
+ val numUsersSubset = userSubset.count()
Seq(2, 4, 6).foreach { k =>
val n = math.min(k, numItems).toInt
@@ -910,13 +910,13 @@
val spark = this.spark
import spark.implicits._
val model = getALSModel
- val numUsers = model.userFactors.count
+ val numUsers = model.userFactors.count()
val expected = Map(
3 -> Seq((0, 54f), (2, 51f), (1, 39f)),
6 -> Seq((0, 28f), (2, 18f), (1, 16f))
)
val itemSubset = expected.keys.toSeq.toDF("item")
- val numItemsSubset = itemSubset.count
+ val numItemsSubset = itemSubset.count()
Seq(2, 3, 4).foreach { k =>
val n = math.min(k, numUsers).toInt
@@ -939,7 +939,7 @@
val singleUserRecs = model.recommendForUserSubset(users, k)
val dupUserRecs = model.recommendForUserSubset(dupUsers, k)
.as[(Int, Seq[(Int, Float)])].collect().toMap
- assert(singleUserRecs.count == dupUserRecs.size)
+ assert(singleUserRecs.count() == dupUserRecs.size)
checkRecommendations(singleUserRecs, dupUserRecs, "item")
val items = Seq(3, 4, 5).toDF("item")
@@ -947,7 +947,7 @@
val singleItemRecs = model.recommendForItemSubset(items, k)
val dupItemRecs = model.recommendForItemSubset(dupItems, k)
.as[(Int, Seq[(Int, Float)])].collect().toMap
- assert(singleItemRecs.count == dupItemRecs.size)
+ assert(singleItemRecs.count() == dupItemRecs.size)
checkRecommendations(singleItemRecs, dupItemRecs, "user")
}
@@ -981,7 +981,7 @@
val spark = this.spark
import spark.implicits._
val (ratings, _) = genExplicitTestData(numUsers = 2, numItems = 2, rank = 1)
- val data = ratings.toDF
+ val data = ratings.toDF()
val model = new ALS()
.setMaxIter(2)
.setImplicitPrefs(true)
@@ -1045,7 +1045,7 @@
// Generate test data
val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0)
// Implicitly test the cleaning of parents during ALS training
- val spark = SparkSession.builder
+ val spark = SparkSession.builder()
.sparkContext(sc)
.getOrCreate()
import spark.implicits._
diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala
index 7e96281..6a745b6 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala
@@ -112,7 +112,7 @@
.setMaxDepth(2)
.setMaxIter(2)
val model = gbt.fit(trainData.toDF())
- testPredictionModelSinglePrediction(model, validationData.toDF)
+ testPredictionModelSinglePrediction(model, validationData.toDF())
}
test("Checkpointing") {
@@ -241,14 +241,14 @@
.setMaxDepth(2)
.setMaxIter(3)
.setLossType(lossType)
- val model3 = gbt.fit(trainData.toDF)
+ val model3 = gbt.fit(trainData.toDF())
val model1 = new GBTRegressionModel("gbt-reg-model-test1",
model3.trees.take(1), model3.treeWeights.take(1), model3.numFeatures)
val model2 = new GBTRegressionModel("gbt-reg-model-test2",
model3.trees.take(2), model3.treeWeights.take(2), model3.numFeatures)
for (evalLossType <- GBTRegressor.supportedLossTypes) {
- val evalArr = model3.evaluateEachIteration(validationData.toDF, evalLossType)
+ val evalArr = model3.evaluateEachIteration(validationData.toDF(), evalLossType)
val lossErr1 = GradientBoostedTrees.computeWeightedError(validationData.map(_.toInstance),
model1.trees, model1.treeWeights, model1.convertToOldLossType(evalLossType))
val lossErr2 = GradientBoostedTrees.computeWeightedError(validationData.map(_.toInstance),
@@ -315,7 +315,7 @@
.setMaxDepth(2)
.setCheckpointInterval(5)
.setSeed(123)
- val model = gbt.fit(trainData.toDF)
+ val model = gbt.fit(trainData.toDF())
model.trees.foreach (i => {
assert(i.getMaxDepth === model.getMaxDepth)
diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala
index 1836b07..df03848 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala
@@ -1719,7 +1719,7 @@
val conf = new SparkConf(false)
val ser = new KryoSerializer(conf).newInstance()
val trainer = new GeneralizedLinearRegression()
- val model = trainer.fit(Seq(Instance(1.0, 1.0, Vectors.dense(1.0, 7.0))).toDF)
+ val model = trainer.fit(Seq(Instance(1.0, 1.0, Vectors.dense(1.0, 7.0))).toDF())
ser.serialize[GeneralizedLinearRegressionModel](model)
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala
index 7981296..411e056 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala
@@ -45,17 +45,17 @@
val rdd = sc.makeRDD(rows)
val df = spark.createDataFrame(rdd, imageSchema)
- assert(df.count === 2, "incorrect image count")
+ assert(df.count() === 2, "incorrect image count")
assert(df.schema("image").dataType == columnSchema, "data do not fit ImageSchema")
}
// TODO(SPARK-40171): Re-enable the following flaky test case after being fixed.
ignore("image datasource count test") {
val df1 = spark.read.format("image").load(imagePath)
- assert(df1.count === 9)
+ assert(df1.count() === 9)
val df2 = spark.read.format("image").option("dropInvalid", true).load(imagePath)
- assert(df2.count === 8)
+ assert(df2.count() === 8)
}
test("image datasource test: read jpg image") {
diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
index 12e9a51..db685b6 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
@@ -173,8 +173,8 @@
test("select features from libsvm relation") {
val df = spark.read.format("libsvm").load(path)
- df.select("features").rdd.map { case Row(d: Vector) => d }.first
- df.select("features").collect
+ df.select("features").rdd.map { case Row(d: Vector) => d }.first()
+ df.select("features").collect()
}
test("create libsvmTable table without schema") {
diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala
index 3ca6816..274e5b0 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala
@@ -96,7 +96,7 @@
Array(6), Gini, QuantileStrategy.Sort,
0, 0, 0.0, 0.0, 0, 0
)
- val featureSamples = Array.fill(10000)((1.0, math.random)).filter(_._2 != 0.0)
+ val featureSamples = Array.fill(10000)((1.0, math.random())).filter(_._2 != 0.0)
val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
assert(splits.length === 5)
assert(fakeMetadata.numSplits(0) === 5)
@@ -183,7 +183,7 @@
Array(6), Gini, QuantileStrategy.Sort,
0, 0, 0.0, 0.0, 0, 0
)
- val featureSamplesUnitWeight = Array.fill(10)((1.0, math.random))
+ val featureSamplesUnitWeight = Array.fill(10)((1.0, math.random()))
val featureSamplesSmallWeight = featureSamplesUnitWeight.map { case (w, x) => (w * 0.001, x)}
val featureSamplesLargeWeight = featureSamplesUnitWeight.map { case (w, x) => (w * 1000, x)}
val splitsUnitWeight = RandomForest
diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/PMMLReadWriteTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/PMMLReadWriteTest.scala
index 19e9fe4..a579684 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/util/PMMLReadWriteTest.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/util/PMMLReadWriteTest.scala
@@ -44,7 +44,7 @@
instance.write.format("pmml").save(path)
}
instance.write.format("pmml").overwrite().save(path)
- val pmmlStr = sc.textFile(path).collect.mkString("\n")
+ val pmmlStr = sc.textFile(path).collect().mkString("\n")
val pmmlModel = PMMLUtils.loadFromString(pmmlStr)
assert(pmmlModel.getHeader().getApplication().getName().startsWith("Apache Spark"))
checkModelData(pmmlModel)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
index c4621c9..ee90c82 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
@@ -143,7 +143,7 @@
for (i <- 0 until nClasses) {
if (p < probs(i)) {
y = i
- break
+ break()
}
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/DistanceMeasureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/DistanceMeasureSuite.scala
index 73691c4..f8a1151 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/DistanceMeasureSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/DistanceMeasureSuite.scala
@@ -40,12 +40,12 @@
val rng = new Random(seed)
centers = Array.tabulate(k) { i =>
- val values = Array.fill(dim)(rng.nextGaussian)
+ val values = Array.fill(dim)(rng.nextGaussian())
new VectorWithNorm(Vectors.dense(values))
}
data = Array.tabulate(1000) { i =>
- val values = Array.fill(dim)(rng.nextGaussian)
+ val values = Array.fill(dim)(rng.nextGaussian())
new VectorWithNorm(Vectors.dense(values))
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
index e63ca70..1737a7f 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
@@ -80,7 +80,8 @@
test("unique cluster centers") {
val rng = new Random(seed)
val numDistinctPoints = 10
- val points = (0 until numDistinctPoints).map(i => Vectors.dense(Array.fill(3)(rng.nextDouble)))
+ val points =
+ (0 until numDistinctPoints).map(i => Vectors.dense(Array.fill(3)(rng.nextDouble())))
val data = sc.parallelize(points.flatMap(Array.fill(1 + rng.nextInt(3))(_)), 2)
val normedData = data.map(new VectorWithNorm(_))
@@ -362,7 +363,7 @@
val n = 200000
val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) =>
val random = new Random(idx)
- iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble)))
+ iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble())))
}.cache()
for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) {
// If we serialize data directly in the task closure, the size of the serialized task would be
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
index ca452a8..19d424c 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
@@ -284,11 +284,11 @@
val nnz = random.nextInt(m)
val indices1 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
- val values1 = Array.fill(nnz)(random.nextDouble)
+ val values1 = Array.fill(nnz)(random.nextDouble())
val sparseVector1 = Vectors.sparse(m, indices1, values1)
val indices2 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
- val values2 = Array.fill(nnz)(random.nextDouble)
+ val values2 = Array.fill(nnz)(random.nextDouble())
val sparseVector2 = Vectors.sparse(m, indices2, values2)
val denseVector1 = Vectors.dense(sparseVector1.toArray)
@@ -578,8 +578,8 @@
valuesBuilder += v
}
val (indices, values) = vec.activeIterator.toArray.unzip
- assert(indicesBuilder.result === indices)
- assert(valuesBuilder.result === values)
+ assert(indicesBuilder.result() === indices)
+ assert(valuesBuilder.result() === values)
}
}
@@ -599,8 +599,8 @@
}
}
val (indices, values) = vec.nonZeroIterator.toArray.unzip
- assert(indicesBuilder.result === indices)
- assert(valuesBuilder.result === values)
+ assert(indicesBuilder.result() === indices)
+ assert(valuesBuilder.result() === values)
}
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala
index 0e78982..bc1ae86 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala
@@ -136,9 +136,9 @@
assert(rowMat.toBreeze() === gridBasedMat.toBreeze())
// SPARK-15922: BlockMatrix to IndexedRowMatrix throws an error"
- val bmat = rowMat.toBlockMatrix
- val imat = bmat.toIndexedRowMatrix
- imat.rows.collect
+ val bmat = rowMat.toBlockMatrix()
+ val imat = bmat.toIndexedRowMatrix()
+ imat.rows.collect()
val rows = 1
val cols = 10
@@ -155,7 +155,7 @@
val rdd = sc.parallelize(vectors)
val B = new BlockMatrix(rdd, rows, cols)
- val C = B.toIndexedRowMatrix.rows.collect
+ val C = B.toIndexedRowMatrix().rows.collect()
(C(0).vector.asBreeze, C(1).vector.asBreeze) match {
case (denseVector: BDV[Double], sparseVector: BSV[Double]) =>
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala
index adc4eee..9bd6529 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala
@@ -293,7 +293,7 @@
val calcR = result.R
assert(closeToZero(abs(expected.q) - abs(calcQ.toBreeze())))
assert(closeToZero(abs(expected.r) - abs(calcR.asBreeze.asInstanceOf[BDM[Double]])))
- assert(closeToZero(calcQ.multiply(calcR).toBreeze - mat.toBreeze()))
+ assert(closeToZero(calcQ.multiply(calcR).toBreeze() - mat.toBreeze()))
// Decomposition without computing Q
val rOnly = mat.tallSkinnyQR(computeQ = false)
assert(rOnly.Q == null)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala
index 1318b23..48c5726 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala
@@ -258,7 +258,7 @@
val n = 200000
val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) =>
val random = new Random(idx)
- iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble))))
+ iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble()))))
}.cache()
val lbfgs = new LBFGS(new LogisticGradient, new SquaredL2Updater)
.setNumCorrections(1)
@@ -268,6 +268,6 @@
val random = new Random(0)
// If we serialize data directly in the task closure, the size of the serialized task would be
// greater than 1MB and hence Spark would throw an error.
- val weights = lbfgs.optimize(examples, Vectors.dense(Array.fill(n)(random.nextDouble)))
+ val weights = lbfgs.optimize(examples, Vectors.dense(Array.fill(n)(random.nextDouble())))
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala
index 60a2781..7d672f7 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala
@@ -35,7 +35,7 @@
// assert that the PMML format is as expected
assert(logisticModelExport.isInstanceOf[PMMLModelExport])
- val pmml = logisticModelExport.getPmml
+ val pmml = logisticModelExport.getPmml()
assert(pmml.getHeader.getDescription === "logistic regression")
// check that the number of fields match the weights size
assert(pmml.getDataDictionary.getNumberOfFields === logisticRegressionModel.weights.size + 1)
@@ -62,7 +62,7 @@
// assert that the PMML format is as expected
assert(svmModelExport.isInstanceOf[PMMLModelExport])
- val pmml = svmModelExport.getPmml
+ val pmml = svmModelExport.getPmml()
assert(pmml.getHeader.getDescription
=== "linear SVM")
// check that the number of fields match the weights size
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala
index bf1a0fd..b6494f3 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala
@@ -32,7 +32,7 @@
val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel)
// assert that the PMML format is as expected
assert(linearModelExport.isInstanceOf[PMMLModelExport])
- val pmml = linearModelExport.getPmml
+ val pmml = linearModelExport.getPmml()
assert(pmml.getHeader.getDescription === "linear regression")
// check that the number of fields match the weights size
assert(pmml.getDataDictionary.getNumberOfFields === linearRegressionModel.weights.size + 1)
@@ -51,7 +51,7 @@
val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel)
// assert that the PMML format is as expected
assert(ridgeModelExport.isInstanceOf[PMMLModelExport])
- val pmml = ridgeModelExport.getPmml
+ val pmml = ridgeModelExport.getPmml()
assert(pmml.getHeader.getDescription === "ridge regression")
// check that the number of fields match the weights size
assert(pmml.getDataDictionary.getNumberOfFields === ridgeRegressionModel.weights.size + 1)
@@ -69,7 +69,7 @@
val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel)
// assert that the PMML format is as expected
assert(lassoModelExport.isInstanceOf[PMMLModelExport])
- val pmml = lassoModelExport.getPmml
+ val pmml = lassoModelExport.getPmml()
assert(pmml.getHeader.getDescription === "lasso regression")
// check that the number of fields match the weights size
assert(pmml.getDataDictionary.getNumberOfFields === lassoModel.weights.size + 1)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala
index 0460b8a..70e6878 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala
@@ -36,7 +36,7 @@
// assert that the PMML format is as expected
assert(modelExport.isInstanceOf[PMMLModelExport])
- val pmml = modelExport.getPmml
+ val pmml = modelExport.getPmml()
assert(pmml.getHeader.getDescription === "k-means clustering")
// check that the number of fields match the single vector size
assert(pmml.getDataDictionary.getNumberOfFields === clusterCenters(0).size)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala
index 470e101..c177ff5 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala
@@ -56,7 +56,7 @@
epsilon: Double = 0.01): Unit = {
assert(expectedNumPartitions === rdd.partitions.size)
val values = new ArrayBuffer[Double]()
- rdd.collect.foreach { vector => {
+ rdd.collect().foreach { vector => {
assert(vector.size === expectedColumns)
values ++= vector.toArray
}}
@@ -144,7 +144,7 @@
// mock distribution to check that partitions have unique seeds
val random = RandomRDDs.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L)
- assert(random.collect.size === random.collect.distinct.size)
+ assert(random.collect().size === random.collect().distinct.size)
}
test("randomVectorRDD for different distributions") {
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala
index 3a7040d..d61514b 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala
@@ -35,7 +35,7 @@
override def beforeAll(): Unit = {
super.beforeAll()
- spark = SparkSession.builder
+ spark = SparkSession.builder()
.master("local[2]")
.appName("MLlibUnitTest")
.getOrCreate()
diff --git a/pom.xml b/pom.xml
index 56ad7f4..da9245a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2972,7 +2972,16 @@
-->
<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>
+ <!-- SPARK-45610 Convert "Auto-application to `()` is deprecated" to compile error, as it will become a compile error in Scala 3. -->
+ <arg>-Wconf:cat=deprecation&msg=Auto-application to \`\(\)\` is deprecated:e</arg>
+ <!--
+ TODO(SPARK-45615): The issue described by https://github.com/scalatest/scalatest/issues/2297 can cause false positives.
+ So SPARK-45610 added the following 4 suppression rules, which can be removed after upgrading scalatest to 3.2.18.
+ -->
+ <arg>-Wconf:cat=deprecation&msg=Auto-application to \`\(\)\` is deprecated&site=org.apache.spark.rdd.RDDSuite:s</arg>
+ <arg>-Wconf:cat=deprecation&msg=Auto-application to \`\(\)\` is deprecated&site=org.apache.spark.scheduler.TaskSetManagerSuite:s</arg>
+ <arg>-Wconf:cat=deprecation&msg=Auto-application to \`\(\)\` is deprecated&site=org.apache.spark.streaming.ReceiverInputDStreamSuite:s</arg>
+ <arg>-Wconf:cat=deprecation&msg=Auto-application to \`\(\)\` is deprecated&site=org.apache.spark.streaming.kafka010.KafkaRDDSuite: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 d2c9771..82a263d 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -237,7 +237,14 @@
// 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",
+ // SPARK-45610 Convert "Auto-application to `()` is deprecated" to compile error, as it will become a compile error in Scala 3.
+ "-Wconf:cat=deprecation&msg=Auto-application to \\`\\(\\)\\` is deprecated:e",
+ // TODO(SPARK-45615): The issue described by https://github.com/scalatest/scalatest/issues/2297 can cause false positives.
+ // So SPARK-45610 added the following 4 suppression rules, which can be removed after upgrading scalatest to 3.2.18.
+ "-Wconf:cat=deprecation&msg=Auto-application to \\`\\(\\)\\` is deprecated&site=org.apache.spark.rdd.RDDSuite:s",
+ "-Wconf:cat=deprecation&msg=Auto-application to \\`\\(\\)\\` is deprecated&site=org.apache.spark.scheduler.TaskSetManagerSuite:s",
+ "-Wconf:cat=deprecation&msg=Auto-application to \\`\\(\\)\\` is deprecated&site=org.apache.spark.streaming.ReceiverInputDStreamSuite:s",
+ "-Wconf:cat=deprecation&msg=Auto-application to \\`\\(\\)\\` is deprecated&site=org.apache.spark.streaming.kafka010.KafkaRDDSuite: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/repl/src/main/scala/org/apache/spark/repl/Main.scala b/repl/src/main/scala/org/apache/spark/repl/Main.scala
index c8d9be4..e3ce28b 100644
--- a/repl/src/main/scala/org/apache/spark/repl/Main.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/Main.scala
@@ -82,7 +82,7 @@
if (!hasErrors) {
interp.run(settings) // Repl starts and goes in loop of R.E.P.L
- Option(sparkContext).foreach(_.stop)
+ Option(sparkContext).foreach(_.stop())
}
}
@@ -103,7 +103,7 @@
conf.setSparkHome(System.getenv("SPARK_HOME"))
}
- val builder = SparkSession.builder.config(conf)
+ val builder = SparkSession.builder().config(conf)
if (conf
.get(CATALOG_IMPLEMENTATION.key, "hive")
.toLowerCase(Locale.ROOT) == "hive") {
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
index 389ba18..662f5dd 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
@@ -198,7 +198,7 @@
// Break the while loop if the pod is completed or we don't want to wait
if (watcher.watchOrStop(sId)) {
watch.close()
- break
+ break()
}
}
}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
index 113e36e..eb9246a 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
@@ -137,7 +137,7 @@
waitForExecutorPodsClock = new ManualClock(0L)
podsAllocatorUnderTest = new ExecutorPodsAllocator(
conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock)
- when(schedulerBackend.getExecutorIds).thenReturn(Seq.empty)
+ when(schedulerBackend.getExecutorIds()).thenReturn(Seq.empty)
podsAllocatorUnderTest.start(TEST_SPARK_APP_ID, schedulerBackend)
when(kubernetesClient.persistentVolumeClaims()).thenReturn(persistentVolumeClaims)
when(persistentVolumeClaims.inNamespace("default")).thenReturn(pvcWithNamespace)
@@ -504,7 +504,7 @@
// Newly created executors (both acknowledged and not) are cleaned up.
waitForExecutorPodsClock.advance(executorIdleTimeout * 2)
- when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "3", "4"))
+ when(schedulerBackend.getExecutorIds()).thenReturn(Seq("1", "3", "4"))
snapshotsStore.notifySubscribers()
// SPARK-34361: even as 1, 3 and 4 are not timed out as they are considered as known PODs so
// this is why they are not counted into the outstanding PODs and /they are not removed even
@@ -586,7 +586,7 @@
verify(podsWithNamespace).resource(podWithAttachedContainerForId(7, rp.id))
// 1) make 1 POD known by the scheduler backend for each resource profile
- when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "4"))
+ when(schedulerBackend.getExecutorIds()).thenReturn(Seq("1", "4"))
snapshotsStore.notifySubscribers()
assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5,
"scheduler backend known PODs are not outstanding")
@@ -594,7 +594,7 @@
// 2) make 1 extra POD known by the scheduler backend for each resource profile
// and make some to pending
- when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "2", "4", "5"))
+ when(schedulerBackend.getExecutorIds()).thenReturn(Seq("1", "2", "4", "5"))
snapshotsStore.updatePod(pendingExecutor(2, defaultProfile.id))
snapshotsStore.updatePod(pendingExecutor(3, defaultProfile.id))
snapshotsStore.updatePod(pendingExecutor(5, rp.id))
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/StatefulSetAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/StatefulSetAllocatorSuite.scala
index f74d2c9..474d5b0 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/StatefulSetAllocatorSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/StatefulSetAllocatorSuite.scala
@@ -124,7 +124,7 @@
snapshotsStore = new DeterministicExecutorPodsSnapshotsStore()
podsAllocatorUnderTest = new StatefulSetPodsAllocator(
conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, null)
- when(schedulerBackend.getExecutorIds).thenReturn(Seq.empty)
+ when(schedulerBackend.getExecutorIds()).thenReturn(Seq.empty)
podsAllocatorUnderTest.start(TEST_SPARK_APP_ID, schedulerBackend)
}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala
index 36c2711..0d73cb8 100644
--- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala
@@ -26,7 +26,7 @@
override def initialize(): Unit = {
Minikube.logVersion()
- val minikubeStatus = Minikube.getMinikubeStatus
+ val minikubeStatus = Minikube.getMinikubeStatus()
require(minikubeStatus == MinikubeStatus.RUNNING,
s"Minikube must be running to use the Minikube backend for integration tests." +
s" Current status is: $minikubeStatus.")
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 4fa7b66..4f1ba3b 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -104,7 +104,7 @@
@volatile private var exitCode = 0
@volatile private var unregistered = false
@volatile private var finished = false
- @volatile private var finalStatus = getDefaultFinalStatus
+ @volatile private var finalStatus = getDefaultFinalStatus()
@volatile private var finalMsg: String = ""
@volatile private var userClassThread: Thread = _
@@ -515,7 +515,7 @@
val driverRef = rpcEnv.setupEndpointRef(
RpcAddress(host, port),
YarnSchedulerBackend.ENDPOINT_NAME)
- createAllocator(driverRef, userConf, rpcEnv, appAttemptId, distCacheConf)
+ createAllocator(driverRef, userConf, rpcEnv, appAttemptId, distCacheConf())
} else {
// Sanity check; should never happen in normal operation, since sc should only be null
// if the user app did not create a SparkContext.
@@ -553,7 +553,7 @@
YarnSchedulerBackend.ENDPOINT_NAME)
addAmIpFilter(Some(driverRef),
System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV))
- createAllocator(driverRef, sparkConf, rpcEnv, appAttemptId, distCacheConf)
+ createAllocator(driverRef, sparkConf, rpcEnv, appAttemptId, distCacheConf())
// In client mode the actor will stop the reporter thread.
reporterThread.join()
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index b209149..8153ed5 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -1187,7 +1187,7 @@
Thread.sleep(interval)
val report: ApplicationReport =
try {
- getApplicationReport
+ getApplicationReport()
} catch {
case e: ApplicationNotFoundException =>
logError(s"Application $appId not found.")
@@ -1353,7 +1353,7 @@
def run(): Unit = {
submitApplication()
if (!launcherBackend.isConnected() && fireAndForget) {
- val report = getApplicationReport
+ val report = getApplicationReport()
val state = report.getYarnApplicationState
logInfo(s"Application report for $appId (state: $state)")
logInfo(formatReportDetails(report, getDriverLogsLink(report)))
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala
index e778254..d7bdd17 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala
@@ -144,5 +144,5 @@
allocatorExcludedNodeList.retain { (_, expiryTime) => expiryTime > now }
}
- refreshExcludedNodes
+ refreshExcludedNodes()
}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
index 3728c33..3e25a3a 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
@@ -28,7 +28,7 @@
extends YarnSchedulerBackend(scheduler, sc) {
override def start(): Unit = {
- val attemptId = ApplicationMaster.getAttemptId
+ val attemptId = ApplicationMaster.getAttemptId()
bindToYarn(attemptId.getApplicationId(), Some(attemptId))
super.start()
totalExpectedExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(sc.conf)
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
index 34848a7..1cfb3d9 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
@@ -138,7 +138,7 @@
override def applicationId(): String = {
appId.map(_.toString).getOrElse {
logWarning("Application ID is not initialized yet.")
- super.applicationId
+ super.applicationId()
}
}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
index 0cabef7..b9f60d6 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
@@ -51,7 +51,7 @@
handler.updateResourceRequests()
handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
- ResourceProfile.clearDefaultProfile
+ ResourceProfile.clearDefaultProfile()
val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf)
val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
3, 15, Map("host3" -> 15, "host4" -> 15, "host5" -> 10),
@@ -76,7 +76,7 @@
createContainer("host2")
))
- ResourceProfile.clearDefaultProfile
+ ResourceProfile.clearDefaultProfile()
val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf)
val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
@@ -100,7 +100,7 @@
createContainer("host2")
))
- ResourceProfile.clearDefaultProfile
+ ResourceProfile.clearDefaultProfile()
val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf)
val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
@@ -122,7 +122,7 @@
createContainer("host3")
))
- ResourceProfile.clearDefaultProfile
+ ResourceProfile.clearDefaultProfile()
val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf)
val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
@@ -138,7 +138,7 @@
handler.updateResourceRequests()
handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
- ResourceProfile.clearDefaultProfile
+ ResourceProfile.clearDefaultProfile()
val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf)
val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
1, 0, Map.empty,
@@ -160,7 +160,7 @@
createContainerRequest(Array("host2", "host3")),
createContainerRequest(Array("host1", "host4")))
- ResourceProfile.clearDefaultProfile
+ ResourceProfile.clearDefaultProfile()
val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf)
val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 2637b2e..806efd39 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -602,7 +602,7 @@
assert(configFromExecutors.find(_ == null) === None)
// verify log urls are present
- val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo]
+ val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo]()
assert(listeners.size === 1)
val listener = listeners(0)
val executorInfos = listener.addedExecutorInfos.values
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala
index a41d2a0..34d19bb 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala
@@ -77,7 +77,7 @@
override def validatePatternString(): Unit = {
try {
formatter
- } catch checkLegacyFormatter(pattern, legacyFormatter.validatePatternString)
+ } catch checkLegacyFormatter(pattern, legacyFormatter.validatePatternString())
()
}
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
index 55eee41..df146e0 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
@@ -316,7 +316,7 @@
*/
class FractionTimestampFormatter(zoneId: ZoneId)
extends Iso8601TimestampFormatter(
- TimestampFormatter.defaultPattern,
+ TimestampFormatter.defaultPattern(),
zoneId,
TimestampFormatter.defaultLocale,
LegacyDateFormats.FAST_DATE_FORMAT,
@@ -510,7 +510,7 @@
isParsing: Boolean,
forTimestampNTZ: Boolean = false): TimestampFormatter = {
val formatter = if (SqlApiConf.get.legacyTimeParserPolicy == LEGACY && !forTimestampNTZ) {
- getLegacyFormatter(format.getOrElse(defaultPattern), zoneId, locale, legacyFormat)
+ getLegacyFormatter(format.getOrElse(defaultPattern()), zoneId, locale, legacyFormat)
} else {
format
.map(new Iso8601TimestampFormatter(_, zoneId, locale, legacyFormat, isParsing))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index f9b0837..0469fb2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -322,7 +322,7 @@
RewriteUpdateTable ::
RewriteMergeIntoTable ::
BindParameters ::
- typeCoercionRules ++
+ typeCoercionRules() ++
Seq(
ResolveWithCTE,
ExtractDistributedSequenceID) ++
@@ -1400,7 +1400,7 @@
cast.setTagValue(Cast.BY_TABLE_INSERTION, ())
Some(Alias(cast, col.name)())
case _ if queryColumns.hasNext =>
- Some(queryColumns.next)
+ Some(queryColumns.next())
case _ =>
None
}
@@ -2343,13 +2343,13 @@
u.filter match {
case Some(filter) if !filter.deterministic =>
- throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
+ throw QueryCompilationErrors.nonDeterministicFilterInAggregateError()
case Some(filter) if filter.dataType != BooleanType =>
- throw QueryCompilationErrors.nonBooleanFilterInAggregateError
+ throw QueryCompilationErrors.nonBooleanFilterInAggregateError()
case Some(filter) if filter.exists(_.isInstanceOf[AggregateExpression]) =>
- throw QueryCompilationErrors.aggregateInAggregateFilterError
+ throw QueryCompilationErrors.aggregateInAggregateFilterError()
case Some(filter) if filter.exists(_.isInstanceOf[WindowExpression]) =>
- throw QueryCompilationErrors.windowFunctionInAggregateFilterError
+ throw QueryCompilationErrors.windowFunctionInAggregateFilterError()
case _ =>
}
if (u.ignoreNulls) {
@@ -3063,7 +3063,7 @@
wsc.copy(partitionSpec = newPartitionSpec, orderSpec = newOrderSpec)
case WindowExpression(ae: AggregateExpression, _) if ae.filter.isDefined =>
- throw QueryCompilationErrors.windowAggregateFunctionWithFilterNotSupportedError
+ throw QueryCompilationErrors.windowAggregateFunctionWithFilterNotSupportedError()
// Extract Windowed AggregateExpression
case we @ WindowExpression(
@@ -3076,7 +3076,7 @@
WindowExpression(newAgg, spec)
case AggregateExpression(aggFunc, _, _, _, _) if hasWindowFunction(aggFunc.children) =>
- throw QueryCompilationErrors.windowFunctionInsideAggregateFunctionNotAllowedError
+ throw QueryCompilationErrors.windowFunctionInsideAggregateFunctionNotAllowedError()
// Extracts AggregateExpression. For example, for SUM(x) - Sum(y) OVER (...),
// we need to extract SUM(x).
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
index 90e8242..a9bbda5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
@@ -126,7 +126,7 @@
dbDefinition, e)
}
val newDb = dbDefinition.copy(
- properties = dbDefinition.properties ++ Map(PROP_OWNER -> Utils.getCurrentUserName))
+ properties = dbDefinition.properties ++ Map(PROP_OWNER -> Utils.getCurrentUserName()))
catalog.put(dbDefinition.name, new DatabaseDesc(newDb))
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index 392c911..f48ff23 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -370,7 +370,7 @@
validateLocation: Boolean = true): Unit = {
val isExternal = tableDefinition.tableType == CatalogTableType.EXTERNAL
if (isExternal && tableDefinition.storage.locationUri.isEmpty) {
- throw QueryCompilationErrors.createExternalTableWithoutLocationError
+ throw QueryCompilationErrors.createExternalTableWithoutLocationError()
}
val qualifiedIdent = qualifyIdentifier(tableDefinition.identifier)
@@ -1124,7 +1124,7 @@
* updated.
*/
def refreshTable(name: TableIdentifier): Unit = synchronized {
- getLocalOrGlobalTempView(name).map(_.refresh).getOrElse {
+ getLocalOrGlobalTempView(name).map(_.refresh()).getOrElse {
val qualifiedIdent = qualifyIdentifier(name)
val qualifiedTableName = QualifiedTableName(qualifiedIdent.database.get, qualifiedIdent.table)
tableRelationCache.invalidate(qualifiedTableName)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
index 29c9605..41071d0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
@@ -65,7 +65,7 @@
case m: CreateMap =>
throw QueryCompilationErrors.keyValueInMapNotStringError(m)
case _ =>
- throw QueryCompilationErrors.nonMapFunctionNotAllowedError
+ throw QueryCompilationErrors.nonMapFunctionNotAllowedError()
}
/**
@@ -78,7 +78,7 @@
schema.getFieldIndex(columnNameOfCorruptRecord).foreach { corruptFieldIndex =>
val f = schema(corruptFieldIndex)
if (f.dataType != StringType || !f.nullable) {
- throw QueryCompilationErrors.invalidFieldTypeForCorruptRecordError
+ throw QueryCompilationErrors.invalidFieldTypeForCorruptRecordError()
}
}
}
@@ -93,7 +93,7 @@
val pos = new ParsePosition(0)
val result = decimalFormat.parse(s, pos).asInstanceOf[java.math.BigDecimal]
if (pos.getIndex() != s.length() || pos.getErrorIndex() != -1) {
- throw QueryExecutionErrors.cannotParseDecimalError
+ throw QueryExecutionErrors.cannotParseDecimalError()
} else {
result
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala
index 1eac386..43bf845 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala
@@ -66,7 +66,7 @@
acceptAny[ArrayData](array => {
val builder = new UTF8StringBuilder
builder.append("[")
- if (array.numElements > 0) {
+ if (array.numElements() > 0) {
val toUTF8String = castToString(et)
if (array.isNullAt(0)) {
if (nullString.nonEmpty) builder.append(nullString)
@@ -74,7 +74,7 @@
builder.append(toUTF8String(array.get(0, et)).asInstanceOf[UTF8String])
}
var i = 1
- while (i < array.numElements) {
+ while (i < array.numElements()) {
builder.append(",")
if (array.isNullAt(i)) {
if (nullString.nonEmpty) builder.append(" " + nullString)
@@ -92,7 +92,7 @@
acceptAny[MapData](map => {
val builder = new UTF8StringBuilder
builder.append(leftBracket)
- if (map.numElements > 0) {
+ if (map.numElements() > 0) {
val keyArray = map.keyArray()
val valueArray = map.valueArray()
val keyToUTF8String = castToString(kt)
@@ -106,7 +106,7 @@
builder.append(valueToUTF8String(valueArray.get(0, vt)).asInstanceOf[UTF8String])
}
var i = 1
- while (i < map.numElements) {
+ while (i < map.numElements()) {
builder.append(", ")
builder.append(keyToUTF8String(keyArray.get(i, kt)).asInstanceOf[UTF8String])
builder.append(" ->")
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala
index bffec27..f304b43 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala
@@ -149,6 +149,6 @@
case Literal(d: Double, DoubleType) => d
case Literal(dec: Decimal, _) => dec.toDouble
case _ =>
- throw QueryCompilationErrors.secondArgumentNotDoubleLiteralError
+ throw QueryCompilationErrors.secondArgumentNotDoubleLiteralError()
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
index d0d4ca6..bb3460e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
@@ -170,7 +170,7 @@
override def eval(buffer: mutable.HashSet[Any]): Any = {
val array = child.dataType match {
case BinaryType =>
- buffer.iterator.map(_.asInstanceOf[ArrayData].toByteArray).toArray
+ buffer.iterator.map(_.asInstanceOf[ArrayData].toByteArray()).toArray
case _ => buffer.toArray
}
new GenericArrayData(array)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 6b54853..3595e43 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -502,7 +502,7 @@
inlineToOuterClass: Boolean): NewFunctionSpec = {
val (className, classInstance) = if (inlineToOuterClass) {
outerClassName -> ""
- } else if (currClassSize > GENERATED_CLASS_SIZE_THRESHOLD) {
+ } else if (currClassSize() > GENERATED_CLASS_SIZE_THRESHOLD) {
val className = freshName("NestedClass")
val classInstance = freshName("nestedClassInstance")
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala
index 3651dc4..552c179 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala
@@ -255,19 +255,19 @@
val inputs = args.iterator
val buf = new StringBuilder(Block.CODE_BLOCK_BUFFER_LENGTH)
- buf.append(StringContext.treatEscapes(strings.next))
+ buf.append(StringContext.treatEscapes(strings.next()))
while (strings.hasNext) {
- val input = inputs.next
+ val input = inputs.next()
input match {
case _: ExprValue | _: CodeBlock =>
codeParts += buf.toString
- buf.clear
+ buf.clear()
blockInputs += input.asInstanceOf[JavaCode]
case EmptyBlock =>
case _ =>
buf.append(input)
}
- buf.append(StringContext.treatEscapes(strings.next))
+ buf.append(StringContext.treatEscapes(strings.next()))
}
codeParts += buf.toString
@@ -291,10 +291,10 @@
val strings = codeParts.iterator
val inputs = blockInputs.iterator
val buf = new StringBuilder(Block.CODE_BLOCK_BUFFER_LENGTH)
- buf.append(strings.next)
+ buf.append(strings.next())
while (strings.hasNext) {
- buf.append(inputs.next)
- buf.append(strings.next)
+ buf.append(inputs.next())
+ buf.append(strings.next())
}
buf.toString
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
index e22af21..3885a5b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
@@ -268,7 +268,7 @@
if (index >= baseValue.numElements() || index < 0) {
if (failOnError) {
throw QueryExecutionErrors.invalidArrayIndexError(
- index, baseValue.numElements, getContextOrNull())
+ index, baseValue.numElements(), getContextOrNull())
} else {
null
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 30a6bec..50a9dbf 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -49,7 +49,7 @@
childrenResolved && checkInputDataTypes().isSuccess && timeZoneId.isDefined
final override val nodePatterns: Seq[TreePattern] =
- Seq(TIME_ZONE_AWARE_EXPRESSION) ++ nodePatternsInternal
+ Seq(TIME_ZONE_AWARE_EXPRESSION) ++ nodePatternsInternal()
// Subclasses can override this function to provide more TreePatterns.
def nodePatternsInternal(): Seq[TreePattern] = Seq()
@@ -1017,7 +1017,7 @@
copy(timeZoneId = Option(timeZoneId))
def this(time: Expression) = {
- this(time, Literal(TimestampFormatter.defaultPattern))
+ this(time, Literal(TimestampFormatter.defaultPattern()))
}
override def prettyName: String = "to_unix_timestamp"
@@ -1073,7 +1073,7 @@
copy(timeZoneId = Option(timeZoneId))
def this(time: Expression) = {
- this(time, Literal(TimestampFormatter.defaultPattern))
+ this(time, Literal(TimestampFormatter.defaultPattern()))
}
def this() = {
@@ -1409,7 +1409,7 @@
override def prettyName: String = "from_unixtime"
def this(unix: Expression) = {
- this(unix, Literal(TimestampFormatter.defaultPattern))
+ this(unix, Literal(TimestampFormatter.defaultPattern()))
}
override def dataType: DataType = StringType
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 4df6a5e..3750a92 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
@@ -336,9 +336,9 @@
override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = {
val arr = argumentValue.asInstanceOf[ArrayData]
val f = functionForEval
- val result = new GenericArrayData(new Array[Any](arr.numElements))
+ val result = new GenericArrayData(new Array[Any](arr.numElements()))
var i = 0
- while (i < arr.numElements) {
+ while (i < arr.numElements()) {
elementVar.value.set(arr.get(i, elementVar.dataType))
if (indexVar.isDefined) {
indexVar.get.value.set(i)
@@ -603,9 +603,9 @@
override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = {
val arr = argumentValue.asInstanceOf[ArrayData]
val f = functionForEval
- val buffer = new mutable.ArrayBuffer[Any](arr.numElements)
+ val buffer = new mutable.ArrayBuffer[Any](arr.numElements())
var i = 0
- while (i < arr.numElements) {
+ while (i < arr.numElements()) {
elementVar.value.set(arr.get(i, elementVar.dataType))
if (indexVar.isDefined) {
indexVar.get.value.set(i)
@@ -683,7 +683,7 @@
var exists = false
var foundNull = false
var i = 0
- while (i < arr.numElements && !exists) {
+ while (i < arr.numElements() && !exists) {
elementVar.value.set(arr.get(i, elementVar.dataType))
val ret = f.eval(inputRow)
if (ret == null) {
@@ -764,7 +764,7 @@
var forall = true
var foundNull = false
var i = 0
- while (i < arr.numElements && forall) {
+ while (i < arr.numElements() && forall) {
elementVar.value.set(arr.get(i, elementVar.dataType))
val ret = f.eval(inputRow)
if (ret == null) {
@@ -934,9 +934,9 @@
override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = {
val map = argumentValue.asInstanceOf[MapData]
- val resultKeys = new GenericArrayData(new Array[Any](map.numElements))
+ val resultKeys = new GenericArrayData(new Array[Any](map.numElements()))
var i = 0
- while (i < map.numElements) {
+ while (i < map.numElements()) {
keyVar.value.set(map.keyArray().get(i, keyVar.dataType))
valueVar.value.set(map.valueArray().get(i, valueVar.dataType))
val result = InternalRow.copyValue(functionForEval.eval(inputRow))
@@ -986,9 +986,9 @@
override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = {
val map = argumentValue.asInstanceOf[MapData]
- val resultValues = new GenericArrayData(new Array[Any](map.numElements))
+ val resultValues = new GenericArrayData(new Array[Any](map.numElements()))
var i = 0
- while (i < map.numElements) {
+ while (i < map.numElements()) {
keyVar.value.set(map.keyArray().get(i, keyVar.dataType))
valueVar.value.set(map.valueArray().get(i, valueVar.dataType))
val v = InternalRow.copyValue(functionForEval.eval(inputRow))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
index 038e7ff..c320d98 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
@@ -1558,25 +1558,25 @@
case ByteType if ansiEnabled =>
MathUtils.withOverflow(
f = BigDecimal(input1.asInstanceOf[Byte]).setScale(_scale, mode).toByteExact,
- context = getContextOrNull)
+ context = getContextOrNull())
case ByteType =>
BigDecimal(input1.asInstanceOf[Byte]).setScale(_scale, mode).toByte
case ShortType if ansiEnabled =>
MathUtils.withOverflow(
f = BigDecimal(input1.asInstanceOf[Short]).setScale(_scale, mode).toShortExact,
- context = getContextOrNull)
+ context = getContextOrNull())
case ShortType =>
BigDecimal(input1.asInstanceOf[Short]).setScale(_scale, mode).toShort
case IntegerType if ansiEnabled =>
MathUtils.withOverflow(
f = BigDecimal(input1.asInstanceOf[Int]).setScale(_scale, mode).toIntExact,
- context = getContextOrNull)
+ context = getContextOrNull())
case IntegerType =>
BigDecimal(input1.asInstanceOf[Int]).setScale(_scale, mode).toInt
case LongType if ansiEnabled =>
MathUtils.withOverflow(
f = BigDecimal(input1.asInstanceOf[Long]).setScale(_scale, mode).toLongExact,
- context = getContextOrNull)
+ context = getContextOrNull())
case LongType =>
BigDecimal(input1.asInstanceOf[Long]).setScale(_scale, mode).toLong
case FloatType =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
index fdebca7..91bd6a6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -811,7 +811,7 @@
override lazy val resolved = false
override def dataType: DataType = customCollectionCls.map(ObjectType.apply).getOrElse {
- throw QueryExecutionErrors.customCollectionClsNotResolvedError
+ throw QueryExecutionErrors.customCollectionClsNotResolvedError()
}
override protected def withNewChildInternal(newChild: Expression): UnresolvedMapObjects =
@@ -1461,7 +1461,7 @@
keys(i) = if (key != null) {
keyConverter.eval(rowWrapper(key))
} else {
- throw QueryExecutionErrors.nullAsMapKeyNotAllowedError
+ throw QueryExecutionErrors.nullAsMapKeyNotAllowedError()
}
values(i) = if (value != null) {
valueConverter.eval(rowWrapper(value))
@@ -1483,7 +1483,7 @@
keys(i) = if (key != null) {
keyConverter.eval(rowWrapper(key))
} else {
- throw QueryExecutionErrors.nullAsMapKeyNotAllowedError
+ throw QueryExecutionErrors.nullAsMapKeyNotAllowedError()
}
values(i) = if (value != null) {
valueConverter.eval(rowWrapper(value))
@@ -1898,7 +1898,7 @@
override def eval(input: InternalRow): Any = {
val inputRow = child.eval(input).asInstanceOf[Row]
if (inputRow == null) {
- throw QueryExecutionErrors.inputExternalRowCannotBeNullError
+ throw QueryExecutionErrors.inputExternalRowCannotBeNullError()
}
if (inputRow.isNullAt(index)) {
throw QueryExecutionErrors.fieldCannotBeNullError(index, fieldName)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index 6aa949b..eff63bf 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -2565,10 +2565,10 @@
var default: Expression = Literal.create(null, StringType)
val branches = ArrayBuffer.empty[(Expression, Expression)]
while (itr.hasNext) {
- val search = itr.next
+ val search = itr.next()
if (itr.hasNext) {
val condition = EqualNullSafe(input, search)
- branches += ((condition, itr.next))
+ branches += ((condition, itr.next()))
} else {
default = search
}
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 9d8b5de..79388cf 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
@@ -41,7 +41,7 @@
bits
}
- final override val nodePatterns: Seq[TreePattern] = Seq(PLAN_EXPRESSION) ++ nodePatternsInternal
+ final override val nodePatterns: Seq[TreePattern] = Seq(PLAN_EXPRESSION) ++ nodePatternsInternal()
override lazy val deterministic: Boolean = children.forall(_.deterministic) &&
plan.deterministic
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index a570962..ca3c9b0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -62,7 +62,7 @@
checkInputDataTypes().isSuccess
override def nullable: Boolean = true
- override def dataType: DataType = throw QueryCompilationErrors.dataTypeOperationUnsupportedError
+ override def dataType: DataType = throw QueryCompilationErrors.dataTypeOperationUnsupportedError()
override def checkInputDataTypes(): TypeCheckResult = {
frameSpecification match {
@@ -182,7 +182,7 @@
* Represents a window frame.
*/
sealed trait WindowFrame extends Expression with Unevaluable {
- override def dataType: DataType = throw QueryCompilationErrors.dataTypeOperationUnsupportedError
+ override def dataType: DataType = throw QueryCompilationErrors.dataTypeOperationUnsupportedError()
override def nullable: Boolean = false
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala
index 0d5974a..01de1e3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala
@@ -152,5 +152,5 @@
/**
* Reset states of all predicates by re-initializing reference counters.
*/
- override def reset(): Unit = predicates.foreach(_.foreach(_.reset))
+ override def reset(): Unit = predicates.foreach(_.foreach(_.reset()))
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 7e0aafc..2b5f542 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -1450,7 +1450,7 @@
val seed = if (ctx.seed != null) {
ctx.seed.getText.toLong
} else {
- (math.random * 1000).toLong
+ (math.random() * 1000).toLong
}
ctx.sampleMethod() match {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index aee4790..36bdc4c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -67,7 +67,7 @@
// Propagate expressions' pattern bits
val exprIterator = expressions.iterator
while (exprIterator.hasNext) {
- bits.union(exprIterator.next.treePatternBits)
+ bits.union(exprIterator.next().treePatternBits)
}
bits
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index fbc4e41..4290a24 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -1332,9 +1332,9 @@
// grouping expression or null, so here we create new instance of it.
val output = if (hasDuplicateGroupingSets) {
val gpos = AttributeReference("_gen_grouping_pos", IntegerType, false)()
- child.output ++ groupByAttrs.map(_.newInstance) :+ gid :+ gpos
+ child.output ++ groupByAttrs.map(_.newInstance()) :+ gid :+ gpos
} else {
- child.output ++ groupByAttrs.map(_.newInstance) :+ gid
+ child.output ++ groupByAttrs.map(_.newInstance()) :+ gid
}
Expand(projections, output, Project(child.output ++ groupByAliases, child))
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
index 9d29ca1..d5cd5a9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
@@ -63,7 +63,7 @@
""".stripMargin
}
- logBasedOnLevel(message)
+ logBasedOnLevel(message())
}
}
}
@@ -81,7 +81,7 @@
}
}
- logBasedOnLevel(message)
+ logBasedOnLevel(message())
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatternBits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatternBits.scala
index b133630..ee4ddef 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatternBits.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatternBits.scala
@@ -39,7 +39,7 @@
final def containsAllPatterns(patterns: TreePattern*): Boolean = {
val iterator = patterns.iterator
while (iterator.hasNext) {
- if (!containsPattern(iterator.next)) {
+ if (!containsPattern(iterator.next())) {
return false
}
}
@@ -53,7 +53,7 @@
final def containsAnyPattern(patterns: TreePattern*): Boolean = {
val iterator = patterns.iterator
while (iterator.hasNext) {
- if (containsPattern(iterator.next)) {
+ if (containsPattern(iterator.next())) {
return true
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
index 3646c70..28ae343 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
@@ -46,6 +46,6 @@
case _ => false
}
- override def hashCode: Int = if (obj == null) 0 else obj.hashCode
+ override def hashCode: Int = if (obj == null) 0 else obj.hashCode()
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
index f6d76bd..f94a065 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
@@ -46,9 +46,9 @@
val out = new StringBuilder()
while (in.hasNext) {
- in.next match {
+ in.next() match {
case c1 if c1 == escapeChar && in.hasNext =>
- val c = in.next
+ val c = in.next()
c match {
case '_' | '%' => out ++= Pattern.quote(Character.toString(c))
case c if c == escapeChar => out ++= Pattern.quote(Character.toString(c))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
index 5557957..8843a9f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
@@ -64,7 +64,7 @@
case BucketTransform(numBuckets, col, sortCol) =>
if (bucketSpec.nonEmpty) {
- throw QueryExecutionErrors.unsupportedMultipleBucketTransformsError
+ throw QueryExecutionErrors.unsupportedMultipleBucketTransformsError()
}
if (sortCol.isEmpty) {
bucketSpec = Some(BucketSpec(numBuckets, col.map(_.fieldNames.mkString(".")), Nil))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala
index f0f02c1..e51d650 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala
@@ -215,7 +215,7 @@
if (update.newDefaultValue().nonEmpty) {
Some(field.withCurrentDefaultValue(update.newDefaultValue()))
} else {
- Some(field.clearCurrentDefaultValue)
+ Some(field.clearCurrentDefaultValue())
})
case delete: DeleteColumn =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SQLOpenHashSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SQLOpenHashSet.scala
index ee4dd54..10023c2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SQLOpenHashSet.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SQLOpenHashSet.scala
@@ -68,7 +68,7 @@
handleNull: () => Unit): (ArrayData, Int) => Unit = {
(array: ArrayData, index: Int) =>
if (array.isNullAt(index)) {
- if (!hashSet.containsNull) {
+ if (!hashSet.containsNull()) {
hashSet.addNull()
handleNull()
}
@@ -126,7 +126,7 @@
}
(value: Any) =>
if (isNaN(value)) {
- if (!hashSet.containsNaN) {
+ if (!hashSet.containsNaN()) {
hashSet.addNaN()
handleNaN(valueNaN)
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index e338a5e..e792bd6 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -1466,7 +1466,7 @@
Iterator.empty
}
- implicit val intEncoder = ExpressionEncoder[Int]
+ implicit val intEncoder = ExpressionEncoder[Int]()
val left = testRelation2.select($"e").analyze
val right = testRelation3.select($"e").analyze
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
index 32c9a3a..02d70c6 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
@@ -683,7 +683,7 @@
def func(k: Int, left: Iterator[Int], right: Iterator[Int]): Iterator[Int] = {
Iterator.empty
}
- implicit val intEncoder = ExpressionEncoder[Int]
+ implicit val intEncoder = ExpressionEncoder[Int]()
left.cogroup[Int, Int, Int, Int](
right,
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala
index b77cc4c..2d61f9f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala
@@ -54,7 +54,7 @@
test("nice error message for missing encoder") {
checkError(
exception = intercept[
- SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable1]),
+ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable1]()),
errorClass = "ENCODER_NOT_FOUND",
parameters = Map(
"typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable",
@@ -63,7 +63,7 @@
checkError(
exception = intercept[
- SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable2]),
+ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable2]()),
errorClass = "ENCODER_NOT_FOUND",
parameters = Map(
"typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable",
@@ -72,7 +72,7 @@
checkError(
exception = intercept[
- SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable3]),
+ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable3]()),
errorClass = "ENCODER_NOT_FOUND",
parameters = Map(
"typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable",
@@ -81,7 +81,7 @@
checkError(
exception = intercept[
- SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable4]),
+ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable4]()),
errorClass = "ENCODER_NOT_FOUND",
parameters = Map(
"typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable",
@@ -90,7 +90,7 @@
checkError(
exception = intercept[
- SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable5]),
+ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable5]()),
errorClass = "ENCODER_NOT_FOUND",
parameters = Map(
"typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable",
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
index 7f54987..82238de 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
@@ -52,7 +52,7 @@
}
test("real type doesn't match encoder schema but they are compatible: product") {
- val encoder = ExpressionEncoder[StringLongClass]
+ val encoder = ExpressionEncoder[StringLongClass]()
// int type can be up cast to long type
val attrs1 = Seq($"a".string, $"b".int)
@@ -64,28 +64,28 @@
}
test("real type doesn't match encoder schema but they are compatible: nested product") {
- val encoder = ExpressionEncoder[ComplexClass]
+ val encoder = ExpressionEncoder[ComplexClass]()
val attrs = Seq($"a".int, $"b".struct($"a".int, $"b".long))
testFromRow(encoder, attrs, InternalRow(1, InternalRow(2, 3L)))
}
test("real type doesn't match encoder schema but they are compatible: tupled encoder") {
val encoder = ExpressionEncoder.tuple(
- ExpressionEncoder[StringLongClass],
- ExpressionEncoder[Long])
+ ExpressionEncoder[StringLongClass](),
+ ExpressionEncoder[Long]())
val attrs = Seq($"a".struct($"a".string, $"b".byte), $"b".int)
testFromRow(encoder, attrs, InternalRow(InternalRow(str, 1.toByte), 2))
}
test("real type doesn't match encoder schema but they are compatible: primitive array") {
- val encoder = ExpressionEncoder[PrimitiveArrayClass]
+ val encoder = ExpressionEncoder[PrimitiveArrayClass]()
val attrs = Seq($"arr".array(IntegerType))
val array = new GenericArrayData(Array(1, 2, 3))
testFromRow(encoder, attrs, InternalRow(array))
}
test("the real type is not compatible with encoder schema: primitive array") {
- val encoder = ExpressionEncoder[PrimitiveArrayClass]
+ val encoder = ExpressionEncoder[PrimitiveArrayClass]()
val attrs = Seq($"arr".array(StringType))
checkError(
exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)),
@@ -103,14 +103,14 @@
}
test("real type doesn't match encoder schema but they are compatible: array") {
- val encoder = ExpressionEncoder[ArrayClass]
+ val encoder = ExpressionEncoder[ArrayClass]()
val attrs = Seq($"arr".array(new StructType().add("a", "int").add("b", "int").add("c", "int")))
val array = new GenericArrayData(Array(InternalRow(1, 2, 3)))
testFromRow(encoder, attrs, InternalRow(array))
}
test("real type doesn't match encoder schema but they are compatible: nested array") {
- val encoder = ExpressionEncoder[NestedArrayClass]
+ val encoder = ExpressionEncoder[NestedArrayClass]()
val et = new StructType().add("arr", ArrayType(
new StructType().add("a", "int").add("b", "int").add("c", "int")))
val attrs = Seq($"nestedArr".array(et))
@@ -120,7 +120,7 @@
}
test("the real type is not compatible with encoder schema: non-array field") {
- val encoder = ExpressionEncoder[ArrayClass]
+ val encoder = ExpressionEncoder[ArrayClass]()
val attrs = Seq($"arr".int)
checkError(
exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)),
@@ -129,7 +129,7 @@
}
test("the real type is not compatible with encoder schema: array element type") {
- val encoder = ExpressionEncoder[ArrayClass]
+ val encoder = ExpressionEncoder[ArrayClass]()
val attrs = Seq($"arr".array(new StructType().add("c", "int")))
checkError(
exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)),
@@ -138,7 +138,7 @@
}
test("the real type is not compatible with encoder schema: nested array element type") {
- val encoder = ExpressionEncoder[NestedArrayClass]
+ val encoder = ExpressionEncoder[NestedArrayClass]()
withClue("inner element is not array") {
val attrs = Seq($"nestedArr".array(new StructType().add("arr", "int")))
@@ -159,7 +159,7 @@
}
test("nullability of array type element should not fail analysis") {
- val encoder = ExpressionEncoder[Seq[Int]]
+ val encoder = ExpressionEncoder[Seq[Int]]()
val attrs = $"a".array(IntegerType) :: Nil
// It should pass analysis
@@ -176,7 +176,7 @@
}
test("the real number of fields doesn't match encoder schema: tuple encoder") {
- val encoder = ExpressionEncoder[(String, Long)]
+ val encoder = ExpressionEncoder[(String, Long)]()
{
val attrs = Seq($"a".string, $"b".long, $"c".int)
@@ -198,7 +198,7 @@
}
test("the real number of fields doesn't match encoder schema: nested tuple encoder") {
- val encoder = ExpressionEncoder[(String, (Long, String))]
+ val encoder = ExpressionEncoder[(String, (Long, String))]()
{
val attrs = Seq($"a".string, $"b".struct($"x".long, $"y".string, $"z".int))
@@ -220,13 +220,13 @@
}
test("nested case class can have different number of fields from the real schema") {
- val encoder = ExpressionEncoder[(String, StringIntClass)]
+ val encoder = ExpressionEncoder[(String, StringIntClass)]()
val attrs = Seq($"a".string, $"b".struct($"a".string, $"b".int, $"c".int))
encoder.resolveAndBind(attrs)
}
test("SPARK-28497: complex type is not compatible with string encoder schema") {
- val encoder = ExpressionEncoder[String]
+ val encoder = ExpressionEncoder[String]()
Seq($"a".struct($"x".long), $"a".array(StringType), Symbol("a").map(StringType, StringType))
.foreach { attr =>
@@ -246,7 +246,7 @@
test("throw exception if real type is not compatible with encoder schema") {
val e1 = intercept[AnalysisException] {
- ExpressionEncoder[StringIntClass].resolveAndBind(Seq($"a".string, $"b".long))
+ ExpressionEncoder[StringIntClass]().resolveAndBind(Seq($"a".string, $"b".long))
}
checkError(exception = e1,
errorClass = "CANNOT_UP_CAST_DATATYPE",
@@ -262,7 +262,7 @@
val e2 = intercept[AnalysisException] {
val structType = new StructType().add("a", StringType).add("b", DecimalType.SYSTEM_DEFAULT)
- ExpressionEncoder[ComplexClass].resolveAndBind(Seq($"a".long, $"b".struct(structType)))
+ ExpressionEncoder[ComplexClass]().resolveAndBind(Seq($"a".long, $"b".struct(structType)))
}
checkError(exception = e2,
@@ -280,7 +280,7 @@
}
test("SPARK-31750: eliminate UpCast if child's dataType is DecimalType") {
- val encoder = ExpressionEncoder[Seq[BigDecimal]]
+ val encoder = ExpressionEncoder[Seq[BigDecimal]]()
val attr = Seq(AttributeReference("a", ArrayType(DecimalType(38, 0)))())
// Before SPARK-31750, it will fail because Decimal(38, 0) can not be casted to Decimal(38, 18)
testFromRow(encoder, attr, InternalRow(ArrayData.toArrayData(Array(Decimal(1.0)))))
@@ -302,8 +302,8 @@
private def castSuccess[T: TypeTag, U: TypeTag]: Unit = {
- val from = ExpressionEncoder[T]
- val to = ExpressionEncoder[U]
+ val from = ExpressionEncoder[T]()
+ val to = ExpressionEncoder[U]()
val catalystType = from.schema.head.dataType.simpleString
test(s"cast from $catalystType to ${implicitly[TypeTag[U]].tpe} should success") {
to.resolveAndBind(toAttributes(from.schema))
@@ -311,8 +311,8 @@
}
private def castFail[T: TypeTag, U: TypeTag]: Unit = {
- val from = ExpressionEncoder[T]
- val to = ExpressionEncoder[U]
+ val from = ExpressionEncoder[T]()
+ val to = ExpressionEncoder[U]()
val catalystType = from.schema.head.dataType.simpleString
test(s"cast from $catalystType to ${implicitly[TypeTag[U]].tpe} should fail") {
intercept[AnalysisException](to.resolveAndBind(toAttributes(from.schema)))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
index 8373f53..1c77b87 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
@@ -390,28 +390,28 @@
encodeDecodeTest(
1 -> 10L,
"tuple with 2 flat encoders")(
- ExpressionEncoder.tuple(ExpressionEncoder[Int], ExpressionEncoder[Long]))
+ ExpressionEncoder.tuple(ExpressionEncoder[Int](), ExpressionEncoder[Long]()))
encodeDecodeTest(
(PrimitiveData(1, 1, 1, 1, 1, 1, true), (3, 30L)),
"tuple with 2 product encoders")(
- ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData], ExpressionEncoder[(Int, Long)]))
+ ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData](), ExpressionEncoder[(Int, Long)]()))
encodeDecodeTest(
(PrimitiveData(1, 1, 1, 1, 1, 1, true), 3),
"tuple with flat encoder and product encoder")(
- ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData], ExpressionEncoder[Int]))
+ ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData](), ExpressionEncoder[Int]()))
encodeDecodeTest(
(3, PrimitiveData(1, 1, 1, 1, 1, 1, true)),
"tuple with product encoder and flat encoder")(
- ExpressionEncoder.tuple(ExpressionEncoder[Int], ExpressionEncoder[PrimitiveData]))
+ ExpressionEncoder.tuple(ExpressionEncoder[Int](), ExpressionEncoder[PrimitiveData]()))
encodeDecodeTest(
(1, (10, 100L)),
"nested tuple encoder") {
- val intEnc = ExpressionEncoder[Int]
- val longEnc = ExpressionEncoder[Long]
+ val intEnc = ExpressionEncoder[Int]()
+ val longEnc = ExpressionEncoder[Long]()
ExpressionEncoder.tuple(intEnc, ExpressionEncoder.tuple(intEnc, longEnc))
}
@@ -516,7 +516,7 @@
// test for nested product encoders
{
- val schema = ExpressionEncoder[(Int, (String, Int))].schema
+ val schema = ExpressionEncoder[(Int, (String, Int))]().schema
assert(schema(0).nullable === false)
assert(schema(1).nullable)
assert(schema(1).dataType.asInstanceOf[StructType](0).nullable)
@@ -526,8 +526,8 @@
// test for tupled encoders
{
val schema = ExpressionEncoder.tuple(
- ExpressionEncoder[Int],
- ExpressionEncoder[(String, Int)]).schema
+ ExpressionEncoder[Int](),
+ ExpressionEncoder[(String, Int)]()).schema
assert(schema(0).nullable === false)
assert(schema(1).nullable)
assert(schema(1).dataType.asInstanceOf[StructType](0).nullable)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
index 7ddb92c..b71b426 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
@@ -72,7 +72,7 @@
}
private def prepareEvaluation(expression: Expression): Expression = {
- val serializer = new JavaSerializer(new SparkConf()).newInstance
+ val serializer = new JavaSerializer(new SparkConf()).newInstance()
val resolver = ResolveTimeZone
val expr = resolver.resolveTimeZones(expression)
assert(expr.resolved)
@@ -123,11 +123,11 @@
result.get(i, f.dataType), expected.get(i, f.dataType), f.dataType, f.nullable)
}
case (result: ArrayData, expected: ArrayData) =>
- result.numElements == expected.numElements && {
+ result.numElements() == expected.numElements() && {
val ArrayType(et, cn) = dataType.asInstanceOf[ArrayType]
var isSame = true
var i = 0
- while (isSame && i < result.numElements) {
+ while (isSame && i < result.numElements()) {
isSame = checkResult(result.get(i, et), expected.get(i, et), et, cn)
i += 1
}
@@ -135,8 +135,8 @@
}
case (result: MapData, expected: MapData) =>
val MapType(kt, vt, vcn) = dataType.asInstanceOf[MapType]
- checkResult(result.keyArray, expected.keyArray, ArrayType(kt, false), false) &&
- checkResult(result.valueArray, expected.valueArray, ArrayType(vt, vcn), false)
+ checkResult(result.keyArray(), expected.keyArray(), ArrayType(kt, false), false) &&
+ checkResult(result.valueArray(), expected.valueArray(), ArrayType(vt, vcn), false)
case (result: Double, expected: Double) =>
if (expected.isNaN) result.isNaN else expected == result
case (result: Float, expected: Float) =>
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala
index 3af7cb3..01ecebf 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala
@@ -107,7 +107,7 @@
test("MapObjects should make copies of unsafe-backed data") {
// test UnsafeRow-backed data
- val structEncoder = ExpressionEncoder[Array[Tuple2[java.lang.Integer, java.lang.Integer]]]
+ val structEncoder = ExpressionEncoder[Array[Tuple2[java.lang.Integer, java.lang.Integer]]]()
val structInputRow = InternalRow.fromSeq(Seq(Array((1, 2), (3, 4))))
val structExpected = new GenericArrayData(
Array(InternalRow.fromSeq(Seq(1, 2)), InternalRow.fromSeq(Seq(3, 4))))
@@ -115,7 +115,7 @@
structEncoder.serializer.head, structExpected, structInputRow)
// test UnsafeArray-backed data
- val arrayEncoder = ExpressionEncoder[Array[Array[Int]]]
+ val arrayEncoder = ExpressionEncoder[Array[Array[Int]]]()
val arrayInputRow = InternalRow.fromSeq(Seq(Array(Array(1, 2), Array(3, 4))))
val arrayExpected = new GenericArrayData(
Array(new GenericArrayData(Array(1, 2)), new GenericArrayData(Array(3, 4))))
@@ -123,7 +123,7 @@
arrayEncoder.serializer.head, arrayExpected, arrayInputRow)
// test UnsafeMap-backed data
- val mapEncoder = ExpressionEncoder[Array[Map[Int, Int]]]
+ val mapEncoder = ExpressionEncoder[Array[Map[Int, Int]]]()
val mapInputRow = InternalRow.fromSeq(Seq(Array(
Map(1 -> 100, 2 -> 200), Map(3 -> 300, 4 -> 400))))
val mapExpected = new GenericArrayData(Seq(
@@ -299,7 +299,7 @@
// by scala values instead of catalyst values.
private def checkObjectExprEvaluation(
expression: => Expression, expected: Any, inputRow: InternalRow = EmptyRow): Unit = {
- val serializer = new JavaSerializer(new SparkConf()).newInstance
+ val serializer = new JavaSerializer(new SparkConf()).newInstance()
val resolver = ResolveTimeZone
val expr = resolver.resolveTimeZones(serializer.deserialize(serializer.serialize(expression)))
checkEvaluationWithoutCodegen(expr, expected, inputRow)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
index 33e521e..db2d8c2 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
@@ -232,14 +232,14 @@
}
test("sample estimation") {
- val sample = Sample(0.0, 0.5, withReplacement = false, (math.random * 1000).toLong, plan)
+ val sample = Sample(0.0, 0.5, withReplacement = false, (math.random() * 1000).toLong, plan)
checkStats(sample, Statistics(sizeInBytes = 60, rowCount = Some(5)))
// Child doesn't have rowCount in stats
val childStats = Statistics(sizeInBytes = 120)
val childPlan = DummyLogicalPlan(childStats, childStats)
val sample2 =
- Sample(0.0, 0.11, withReplacement = false, (math.random * 1000).toLong, childPlan)
+ Sample(0.0, 0.11, withReplacement = false, (math.random() * 1000).toLong, childPlan)
checkStats(sample2, Statistics(sizeInBytes = 14))
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala
index 50667c5..d55e672 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala
@@ -27,7 +27,7 @@
class ArrayDataIndexedSeqSuite extends SparkFunSuite {
private def compArray(arrayData: ArrayData, elementDt: DataType, array: Array[Any]): Unit = {
- assert(arrayData.numElements == array.length)
+ assert(arrayData.numElements() == array.length)
array.zipWithIndex.map { case (e, i) =>
if (e != null) {
elementDt match {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
index d2fc89a..977dde1 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
@@ -268,7 +268,7 @@
withClue(s"zoneId = ${zoneId.getId}") {
val formatters = LegacyDateFormats.values.toSeq.map { legacyFormat =>
TimestampFormatter(
- TimestampFormatter.defaultPattern,
+ TimestampFormatter.defaultPattern(),
zoneId,
TimestampFormatter.defaultLocale,
legacyFormat,
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala
index 1801094..d8db7b5 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala
@@ -73,8 +73,8 @@
}
private def toUnsafeArray[T: TypeTag](array: Array[T]): ArrayData = {
- val converted = ExpressionEncoder[Array[T]].createSerializer().apply(array).getArray(0)
- assert(converted.numElements == array.length)
+ val converted = ExpressionEncoder[Array[T]]().createSerializer().apply(array).getArray(0)
+ assert(converted.numElements() == array.length)
converted
}
@@ -134,7 +134,7 @@
val unsafeDecimal = ir.getArray(0)
assert(unsafeDecimal.isInstanceOf[UnsafeArrayData])
- assert(unsafeDecimal.numElements == decimalArray.length)
+ assert(unsafeDecimal.numElements() == decimalArray.length)
decimalArray.zipWithIndex.map { case (e, i) =>
assert(unsafeDecimal.getDecimal(i, e.precision, e.scale).toBigDecimal == e)
}
@@ -146,7 +146,7 @@
val ir = encoder.createSerializer().apply(externalRow)
val unsafeCalendar = ir.getArray(0)
assert(unsafeCalendar.isInstanceOf[UnsafeArrayData])
- assert(unsafeCalendar.numElements == calendarintervalArray.length)
+ assert(unsafeCalendar.numElements() == calendarintervalArray.length)
calendarintervalArray.zipWithIndex.map { case (e, i) =>
assert(unsafeCalendar.getInterval(i) == e)
}
@@ -155,7 +155,7 @@
intMultiDimArray.zipWithIndex.map { case (a, j) =>
val u = unsafeMultiDimInt.getArray(j)
assert(u.isInstanceOf[UnsafeArrayData])
- assert(u.numElements == a.length)
+ assert(u.numElements() == a.length)
a.zipWithIndex.map { case (e, i) =>
assert(u.getInt(i) == e)
}
@@ -165,7 +165,7 @@
doubleMultiDimArray.zipWithIndex.map { case (a, j) =>
val u = unsafeMultiDimDouble.getArray(j)
assert(u.isInstanceOf[UnsafeArrayData])
- assert(u.numElements == a.length)
+ assert(u.numElements() == a.length)
a.zipWithIndex.map { case (e, i) =>
assert(u.getDouble(i) == e)
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala
index a0a4d8b..7765bc2 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala
@@ -534,7 +534,7 @@
protected object TruncateAndAppend extends TestBatchWrite {
override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
- dataMap.clear
+ dataMap.clear()
withData(messages.map(_.asInstanceOf[BufferedRows]))
}
}
@@ -572,7 +572,7 @@
protected object StreamingTruncateAndAppend extends TestStreamingWrite {
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {
dataMap.synchronized {
- dataMap.clear
+ dataMap.clear()
withData(messages.map(_.asInstanceOf[BufferedRows]))
}
}
@@ -656,7 +656,7 @@
private def addMetadata(row: InternalRow): InternalRow = {
val metadataRow = new GenericInternalRow(metadataColumnNames.map {
case "index" => index
- case "_partition" => UTF8String.fromString(partition.keyString)
+ case "_partition" => UTF8String.fromString(partition.keyString())
}.toArray)
new JoinedRow(row, metadataRow)
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala
index b4285f31..f4e3d9e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala
@@ -78,7 +78,7 @@
val matchingKeys =
p.children().drop(1).map(_.asInstanceOf[LiteralValue[_]].value.toString).toSet
data = data.filter(partition => {
- val key = partition.asInstanceOf[BufferedRows].keyString
+ val key = partition.asInstanceOf[BufferedRows].keyString()
matchingKeys.contains(key)
})
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index 3a6fb10..aa3fe8d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -1297,7 +1297,7 @@
}
private def assertOnDriver(): Unit = {
- if (TaskContext.get != null) {
+ if (TaskContext.get() != null) {
// we're accessing it during task execution, fail.
throw new IllegalStateException(
"SparkSession should only be created and accessed on the driver.")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
index 033fe5b..56515ca 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
@@ -57,7 +57,7 @@
ident, "ADD COLUMN with qualified column")
}
if (!c.nullable) {
- throw QueryCompilationErrors.addColumnWithV1TableCannotSpecifyNotNullError
+ throw QueryCompilationErrors.addColumnWithV1TableCannotSpecifyNotNullError()
}
}
AlterTableAddColumnsCommand(ident, cols.map(convertToStructField))
@@ -72,7 +72,7 @@
catalog, ident, "ALTER COLUMN with qualified column")
}
if (a.nullable.isDefined) {
- throw QueryCompilationErrors.alterColumnWithV1TableCannotSpecifyNotNullError
+ throw QueryCompilationErrors.alterColumnWithV1TableCannotSpecifyNotNullError()
}
if (a.position.isDefined) {
throw QueryCompilationErrors.unsupportedTableOperationError(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GroupedIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GroupedIterator.scala
index 431f021..ccf3258 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GroupedIterator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GroupedIterator.scala
@@ -94,7 +94,7 @@
* because we will consume the input data to skip to next group while fetching a new iterator,
* thus make the previous iterator empty.
*/
- def hasNext: Boolean = currentIterator != null || fetchNextGroupIterator
+ def hasNext: Boolean = currentIterator != null || fetchNextGroupIterator()
def next(): (InternalRow, Iterator[InternalRow]) = {
assert(hasNext) // Ensure we have fetched the next iterator.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
index 404c46a..a0dd939 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
@@ -219,7 +219,7 @@
errorOnDuplicatedFieldNames: Boolean): ArrowBatchWithSchemaIterator = {
new ArrowBatchWithSchemaIterator(
rowIter, schema, maxRecordsPerBatch, maxEstimatedBatchSize,
- timeZoneId, errorOnDuplicatedFieldNames, TaskContext.get)
+ timeZoneId, errorOnDuplicatedFieldNames, TaskContext.get())
}
private[sql] def createEmptyArrowBatch(
@@ -228,7 +228,7 @@
errorOnDuplicatedFieldNames: Boolean): Array[Byte] = {
new ArrowBatchWithSchemaIterator(
Iterator.empty, schema, 0L, 0L,
- timeZoneId, errorOnDuplicatedFieldNames, TaskContext.get) {
+ timeZoneId, errorOnDuplicatedFieldNames, TaskContext.get()) {
override def hasNext: Boolean = true
}.next()
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
index d2f65b7..1f47673 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
@@ -80,7 +80,7 @@
if (!row.isNullAt(ordinal)) {
count += 1
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -96,7 +96,7 @@
val value = row.getBoolean(ordinal)
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -120,7 +120,7 @@
val value = row.getByte(ordinal)
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -144,7 +144,7 @@
val value = row.getShort(ordinal)
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -168,7 +168,7 @@
val value = row.getInt(ordinal)
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -192,7 +192,7 @@
val value = row.getLong(ordinal)
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -216,7 +216,7 @@
val value = row.getFloat(ordinal)
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -240,7 +240,7 @@
val value = row.getDouble(ordinal)
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -265,7 +265,7 @@
val size = STRING.actualSize(row, ordinal)
gatherValueStats(value, size)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -287,7 +287,7 @@
sizeInBytes += size
count += 1
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -301,7 +301,7 @@
sizeInBytes += CALENDAR_INTERVAL.actualSize(row, ordinal)
count += 1
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -321,7 +321,7 @@
// TODO: this is not right for DecimalType with precision > 18
gatherValueStats(value)
} else {
- gatherNullStats
+ gatherNullStats()
}
}
@@ -345,7 +345,7 @@
sizeInBytes += size
count += 1
} else {
- gatherNullStats
+ gatherNullStats()
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
index 54e8181..a94140d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
@@ -170,7 +170,7 @@
sparkSession, table, table.storage.locationUri, SaveMode.Append, tableExists = true)
} else {
table.storage.locationUri.foreach { p =>
- DataWritingCommand.assertEmptyRootPath(p, mode, sparkSession.sessionState.newHadoopConf)
+ DataWritingCommand.assertEmptyRootPath(p, mode, sparkSession.sessionState.newHadoopConf())
}
assert(table.schema.isEmpty)
sparkSession.sessionState.catalog.validateTableLocation(table)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
index 7d1acc7..1465e32 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
@@ -393,7 +393,7 @@
if (field.name == originColumn.name) {
// Create a new column from the origin column with the new comment.
val withNewComment: StructField =
- addComment(field, newColumn.getComment)
+ addComment(field, newColumn.getComment())
// Create a new column from the origin column with the new current default value.
if (newColumn.getCurrentDefaultValue().isDefined) {
if (newColumn.getCurrentDefaultValue().get.nonEmpty) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
index a3d2d2e..e9aa6d8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
@@ -173,7 +173,7 @@
}
currentWriter.write(record)
- statsTrackers.foreach(_.newRow(currentWriter.path, record))
+ statsTrackers.foreach(_.newRow(currentWriter.path(), record))
recordsInFile += 1
}
}
@@ -326,7 +326,7 @@
protected def writeRecord(record: InternalRow): Unit = {
val outputRow = getOutputRow(record)
currentWriter.write(outputRow)
- statsTrackers.foreach(_.newRow(currentWriter.path, outputRow))
+ statsTrackers.foreach(_.newRow(currentWriter.path(), outputRow))
recordsInFile += 1
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
index 83064a8..3c57903 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
@@ -71,7 +71,7 @@
override lazy val metadataOutput: Seq[AttributeReference] = relation match {
case relation: HadoopFsRelation =>
- metadataOutputWithOutConflicts(Seq(relation.fileFormat.createFileMetadataCol))
+ metadataOutputWithOutConflicts(Seq(relation.fileFormat.createFileMetadataCol()))
case _ => Nil
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala
index 8a6221c..b325a2f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala
@@ -376,7 +376,7 @@
.named(schemaName)
}
}
- (primitiveTypeBuilder.result, valuesBuilder.result)
+ (primitiveTypeBuilder.result(), valuesBuilder.result())
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
index 67e77a9..8288e7e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
@@ -169,7 +169,7 @@
private class MetricsRowIterator(
iter: Iterator[InternalRow]) extends MetricsIterator[InternalRow](iter) {
override def next(): InternalRow = {
- val item = iter.next
+ val item = iter.next()
metricsHandler.updateMetrics(1)
item
}
@@ -178,7 +178,7 @@
private class MetricsBatchIterator(
iter: Iterator[ColumnarBatch]) extends MetricsIterator[ColumnarBatch](iter) {
override def next(): ColumnarBatch = {
- val batch: ColumnarBatch = iter.next
+ val batch: ColumnarBatch = iter.next()
metricsHandler.updateMetrics(batch.numRows)
batch
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala
index b12f55e..1065d63 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala
@@ -58,7 +58,7 @@
columnsProperties: util.Map[NamedReference, util.Map[String, String]],
properties: util.Map[String, String]): Unit = {
JdbcUtils.withConnection(jdbcOptions) { conn =>
- JdbcUtils.classifyException(s"Failed to create index $indexName in $name",
+ JdbcUtils.classifyException(s"Failed to create index $indexName in ${name()}",
JdbcDialects.get(jdbcOptions.url)) {
JdbcUtils.createIndex(
conn, indexName, ident, columns, columnsProperties, properties, jdbcOptions)
@@ -74,7 +74,7 @@
override def dropIndex(indexName: String): Unit = {
JdbcUtils.withConnection(jdbcOptions) { conn =>
- JdbcUtils.classifyException(s"Failed to drop index $indexName in $name",
+ JdbcUtils.classifyException(s"Failed to drop index $indexName in ${name()}",
JdbcDialects.get(jdbcOptions.url)) {
JdbcUtils.dropIndex(conn, indexName, ident, jdbcOptions)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
index 7c48baf..3ae76a1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
@@ -341,7 +341,7 @@
s"HashJoin should not take $x as the JoinType")
}
- val resultProj = createResultProjection
+ val resultProj = createResultProjection()
joinedIter.map { r =>
numOutputRows += 1
resultProj(r)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala
index a229931..f5c2d9a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala
@@ -74,7 +74,9 @@
private lazy val context = TaskContext.get()
private lazy val buffer = new PythonForeachWriter.UnsafeRowBuffer(
- context.taskMemoryManager, new File(Utils.getLocalDir(SparkEnv.get.conf)), schema.fields.length)
+ context.taskMemoryManager(),
+ new File(Utils.getLocalDir(SparkEnv.get.conf)),
+ schema.fields.length)
private lazy val inputRowIterator = buffer.iterator
private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
@@ -101,7 +103,7 @@
override def open(partitionId: Long, version: Long): Boolean = {
outputIterator // initialize everything
writerThread.start()
- TaskContext.get.addTaskCompletionListener[Unit] { _ => buffer.close() }
+ TaskContext.get().addTaskCompletionListener[Unit] { _ => buffer.close() }
true
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala
index 5009257..99d51f6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala
@@ -122,7 +122,7 @@
input: mutable.Map[Any, Long]): mutable.Map[Any, Long] = {
val otherIter = input.iterator
while (otherIter.hasNext) {
- val (key, count) = otherIter.next
+ val (key, count) = otherIter.next()
add(buffer, key, count)
}
buffer
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala
index dae2b70..e7f1aff 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala
@@ -109,7 +109,7 @@
/** Calculate the Pearson Correlation Coefficient for the given columns */
def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = {
- calculateCorrImpl(df, cols).head.getDouble(0)
+ calculateCorrImpl(df, cols).head().getDouble(0)
}
private[sql] def calculateCorrImpl(
@@ -144,7 +144,7 @@
* @return the covariance of the two columns.
*/
def calculateCov(df: DataFrame, cols: Seq[String]): Double = {
- calculateCovImpl(df, cols).head.getDouble(0)
+ calculateCovImpl(df, cols).head().getDouble(0)
}
private[sql] def calculateCovImpl(df: DataFrame, cols: Seq[String]): DataFrame = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
index ad32128..af2c97b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
@@ -219,8 +219,8 @@
}
private def generateTempPath(path: Path): Path = {
- val tc = org.apache.spark.TaskContext.get
- val tid = if (tc != null) ".TID" + tc.taskAttemptId else ""
+ val tc = org.apache.spark.TaskContext.get()
+ val tid = if (tc != null) ".TID" + tc.taskAttemptId() else ""
new Path(path.getParent, s".${path.getName}.${UUID.randomUUID}${tid}.tmp")
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
index ad7c59b..1d428e6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
@@ -56,8 +56,8 @@
if (!lines.hasNext) {
throw new IllegalStateException("Incomplete log file in the offset commit log")
}
- validateVersion(lines.next.trim, VERSION)
- val metadataJson = if (lines.hasNext) lines.next else EMPTY_JSON
+ validateVersion(lines.next().trim, VERSION)
+ val metadataJson = if (lines.hasNext) lines.next() else EMPTY_JSON
CommitMetadata(metadataJson)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
index c7b0695..a5d114b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
@@ -61,7 +61,7 @@
val metadataPath = new Path(path)
protected val fileManager =
- CheckpointFileManager.create(metadataPath, sparkSession.sessionState.newHadoopConf)
+ CheckpointFileManager.create(metadataPath, sparkSession.sessionState.newHadoopConf())
if (!fileManager.exists(metadataPath)) {
fileManager.mkdirs(metadataPath)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
index a67097f..8f01c99 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
@@ -199,7 +199,7 @@
case StateStoreSaveExec(keys, None, None, None, None, stateFormatVersion,
UnaryExecNode(agg,
StateStoreRestoreExec(_, None, _, child))) =>
- val aggStateInfo = nextStatefulOperationStateInfo
+ val aggStateInfo = nextStatefulOperationStateInfo()
StateStoreSaveExec(
keys,
Some(aggStateInfo),
@@ -218,7 +218,7 @@
stateFormatVersion,
UnaryExecNode(agg,
SessionWindowStateStoreRestoreExec(_, _, None, None, None, _, child))) =>
- val aggStateInfo = nextStatefulOperationStateInfo
+ val aggStateInfo = nextStatefulOperationStateInfo()
SessionWindowStateStoreSaveExec(
keys,
session,
@@ -241,7 +241,7 @@
StreamingDeduplicateExec(
keys,
child,
- Some(nextStatefulOperationStateInfo),
+ Some(nextStatefulOperationStateInfo()),
eventTimeWatermarkForLateEvents = None,
eventTimeWatermarkForEviction = None)
@@ -249,7 +249,7 @@
StreamingDeduplicateWithinWatermarkExec(
keys,
child,
- Some(nextStatefulOperationStateInfo),
+ Some(nextStatefulOperationStateInfo()),
eventTimeWatermarkForLateEvents = None,
eventTimeWatermarkForEviction = None)
@@ -257,7 +257,7 @@
// We set this to true only for the first batch of the streaming query.
val hasInitialState = (currentBatchId == 0L && m.hasInitialState)
m.copy(
- stateInfo = Some(nextStatefulOperationStateInfo),
+ stateInfo = Some(nextStatefulOperationStateInfo()),
batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
eventTimeWatermarkForLateEvents = None,
eventTimeWatermarkForEviction = None,
@@ -266,7 +266,7 @@
case m: FlatMapGroupsInPandasWithStateExec =>
m.copy(
- stateInfo = Some(nextStatefulOperationStateInfo),
+ stateInfo = Some(nextStatefulOperationStateInfo()),
batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
eventTimeWatermarkForLateEvents = None,
eventTimeWatermarkForEviction = None
@@ -274,14 +274,14 @@
case j: StreamingSymmetricHashJoinExec =>
j.copy(
- stateInfo = Some(nextStatefulOperationStateInfo),
+ stateInfo = Some(nextStatefulOperationStateInfo()),
eventTimeWatermarkForLateEvents = None,
eventTimeWatermarkForEviction = None
)
case l: StreamingGlobalLimitExec =>
l.copy(
- stateInfo = Some(nextStatefulOperationStateInfo),
+ stateInfo = Some(nextStatefulOperationStateInfo()),
outputMode = Some(outputMode))
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
index 756ee0c..3febce0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
@@ -280,7 +280,7 @@
if (isActive) {
// check if there are any previous errors and bubble up any existing async operations
- errorNotifier.throwErrorIfExists
+ errorNotifier.throwErrorIfExists()
var currentBatchHasNewData = false // Whether the current batch had new data
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index d9cab24..d9b3b20 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -243,7 +243,7 @@
/** All checkpoint file operations should be performed through `CheckpointFileManager`. */
private val fileManager = CheckpointFileManager.create(new Path(resolvedCheckpointRoot),
- sparkSession.sessionState.newHadoopConf)
+ sparkSession.sessionState.newHadoopConf())
/**
* Starts the execution. This returns only after the thread has started and [[QueryStartedEvent]]
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
index 0870867..135d46c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
@@ -64,7 +64,7 @@
// If the dataSource provided class is a same or subclass of FileFormat class
case f if classOf[FileFormat].isAssignableFrom(f) =>
metadataOutputWithOutConflicts(
- Seq(dataSource.providingInstance().asInstanceOf[FileFormat].createFileMetadataCol))
+ Seq(dataSource.providingInstance().asInstanceOf[FileFormat].createFileMetadataCol()))
case _ => Nil
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala
index e15fc8b..e807471 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala
@@ -62,11 +62,11 @@
override def execute(triggerHandler: () => Boolean): Unit = {
while (true) {
- val triggerTimeMs = clock.getTimeMillis
+ val triggerTimeMs = clock.getTimeMillis()
val nextTriggerTimeMs = nextBatchTime(triggerTimeMs)
val terminated = !triggerHandler()
if (intervalMs > 0) {
- val batchElapsedTimeMs = clock.getTimeMillis - triggerTimeMs
+ val batchElapsedTimeMs = clock.getTimeMillis() - triggerTimeMs
if (batchElapsedTimeMs > intervalMs) {
notifyBatchFallingBehind(batchElapsedTimeMs)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
index 368dfae..d1b346b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
@@ -56,8 +56,8 @@
implicit val defaultFormats: DefaultFormats = DefaultFormats
- private val encoder = ExpressionEncoder.tuple(ExpressionEncoder[String],
- ExpressionEncoder[Timestamp])
+ private val encoder = ExpressionEncoder.tuple(ExpressionEncoder[String](),
+ ExpressionEncoder[Timestamp]())
@GuardedBy("this")
private var socket: Socket = _
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
index 6024955..1e3f3a6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
@@ -559,12 +559,12 @@
}
if (isAcquiredByDifferentThread) {
val stackTraceOutput = acquiredThreadInfo.threadRef.get.get.getStackTrace.mkString("\n")
- throw QueryExecutionErrors.unreleasedThreadError(loggingId, newAcquiredThreadInfo.toString,
- acquiredThreadInfo.toString, timeWaitedMs, stackTraceOutput)
+ throw QueryExecutionErrors.unreleasedThreadError(loggingId, newAcquiredThreadInfo.toString(),
+ acquiredThreadInfo.toString(), timeWaitedMs, stackTraceOutput)
} else {
acquiredThreadInfo = newAcquiredThreadInfo
// Add a listener to always release the lock when the task (if active) completes
- Option(TaskContext.get).foreach(_.addTaskCompletionListener[Unit] { _ => this.release() })
+ Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit] { _ => this.release() })
logInfo(s"RocksDB instance was acquired by $acquiredThreadInfo")
}
}
@@ -898,8 +898,8 @@
override def toString(): String = {
val taskStr = if (tc != null) {
val taskDetails =
- s"partition ${tc.partitionId}.${tc.attemptNumber} in stage " +
- s"${tc.stageId}.${tc.stageAttemptNumber()}, TID ${tc.taskAttemptId}"
+ s"partition ${tc.partitionId()}.${tc.attemptNumber()} in stage " +
+ s"${tc.stageId()}.${tc.stageAttemptNumber()}, TID ${tc.taskAttemptId()}"
s", task: $taskDetails"
} else ""
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
index bda560b..def4a41 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
@@ -897,7 +897,7 @@
val (upserted, deleted) = stateManager.updateSessions(store, curKey, curValuesOnKey.toSeq)
numUpdatedStateRows += upserted
numRemovedStateRows += deleted
- curValuesOnKey.clear
+ curValuesOnKey.clear()
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
index 0de16a4..8a2a528 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
@@ -131,9 +131,9 @@
// Reset the metrics tracking object for the new attempt.
Option(stageMetrics.get(event.stageInfo.stageId)).foreach { stage =>
- if (stage.attemptId != event.stageInfo.attemptNumber) {
+ if (stage.attemptId != event.stageInfo.attemptNumber()) {
stageMetrics.put(event.stageInfo.stageId,
- new LiveStageMetrics(event.stageInfo.stageId, event.stageInfo.attemptNumber,
+ new LiveStageMetrics(event.stageInfo.stageId, event.stageInfo.attemptNumber(),
stage.numTasks, stage.accumIdsToMetricType))
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index 74a4f1c..5650e9d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -314,7 +314,7 @@
isTemporary = true)
case _ =>
- val catalogPath = (currentCatalog +:
+ val catalogPath = (currentCatalog() +:
sparkSession.sessionState.catalogManager.currentNamespace).mkString(".")
throw QueryCompilationErrors.unresolvedRoutineError(ident, Seq(catalogPath), plan.origin)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala
index 3a3246a..8f537aa 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala
@@ -73,7 +73,7 @@
*/
override def getTruncateQuery(
table: String,
- cascade: Option[Boolean] = isCascadingTruncateTable): String = {
+ cascade: Option[Boolean] = isCascadingTruncateTable()): String = {
dialects.head.getTruncateQuery(table, cascade)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala
index 5d56b37..189dedb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala
@@ -107,7 +107,7 @@
// scalastyle:on line.size.limit
override def getTruncateQuery(
table: String,
- cascade: Option[Boolean] = isCascadingTruncateTable): String = {
+ cascade: Option[Boolean] = isCascadingTruncateTable()): String = {
s"TRUNCATE TABLE $table IMMEDIATE"
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala
index 1b71528..697ddd2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala
@@ -88,6 +88,6 @@
while (rs.next()) {
schemaBuilder += Array(rs.getString(1))
}
- schemaBuilder.result
+ schemaBuilder.result()
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
index 37c378c..966092b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
@@ -243,7 +243,7 @@
*/
@Since("2.3.0")
def getTruncateQuery(table: String): String = {
- getTruncateQuery(table, isCascadingTruncateTable)
+ getTruncateQuery(table, isCascadingTruncateTable())
}
/**
@@ -257,7 +257,7 @@
@Since("2.4.0")
def getTruncateQuery(
table: String,
- cascade: Option[Boolean] = isCascadingTruncateTable): String = {
+ cascade: Option[Boolean] = isCascadingTruncateTable()): String = {
s"TRUNCATE TABLE $table"
}
@@ -437,7 +437,7 @@
while (rs.next()) {
schemaBuilder += Array(rs.getString(1))
}
- schemaBuilder.result
+ schemaBuilder.result()
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
index a08c893..3c6d02d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
@@ -125,7 +125,7 @@
case _: Exception =>
logWarning("Cannot show schemas.")
}
- schemaBuilder.result
+ schemaBuilder.result()
}
override def getTableExistsQuery(table: String): String = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala
index 3a0333c..bcc8bc4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala
@@ -146,7 +146,7 @@
*/
override def getTruncateQuery(
table: String,
- cascade: Option[Boolean] = isCascadingTruncateTable): String = {
+ cascade: Option[Boolean] = isCascadingTruncateTable()): String = {
cascade match {
case Some(true) => s"TRUNCATE TABLE $table CASCADE"
case _ => s"TRUNCATE TABLE $table"
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
index 9c1ca2c..e3af807 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
@@ -64,7 +64,7 @@
} else if ("text".equalsIgnoreCase(typeName)) {
Some(StringType) // sqlType is Types.VARCHAR
} else if (sqlType == Types.ARRAY) {
- val scale = md.build.getLong("scale").toInt
+ val scale = md.build().getLong("scale").toInt
// postgres array type names start with underscore
toCatalystType(typeName.drop(1), size, scale).map(ArrayType(_))
} else None
@@ -149,7 +149,7 @@
*/
override def getTruncateQuery(
table: String,
- cascade: Option[Boolean] = isCascadingTruncateTable): String = {
+ cascade: Option[Boolean] = isCascadingTruncateTable()): String = {
cascade match {
case Some(true) => s"TRUNCATE TABLE ONLY $table CASCADE"
case _ => s"TRUNCATE TABLE ONLY $table"
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala
index fb7728d..0f0812b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala
@@ -59,7 +59,7 @@
*/
override def getTruncateQuery(
table: String,
- cascade: Option[Boolean] = isCascadingTruncateTable): String = {
+ cascade: Option[Boolean] = isCascadingTruncateTable()): String = {
s"DELETE FROM $table ALL"
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala
index aae2fad..48a3560 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala
@@ -93,7 +93,7 @@
progressIds.enqueue(getUniqueId(runId, batchId, timestamp))
store.write(new StreamingQueryProgressWrapper(event.progress))
while (progressIds.length > streamingProgressRetention) {
- val uniqueId = progressIds.dequeue
+ val uniqueId = progressIds.dequeue()
store.delete(classOf[StreamingQueryProgressWrapper], uniqueId)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
index 7d41133..2db37745 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
@@ -951,7 +951,7 @@
val cachedData = checkIfNoJobTriggered {
spark.range(1002).filter($"id" > 1000).orderBy($"id".desc).cache()
}
- assert(cachedData.collect === Seq(1001))
+ assert(cachedData.collect() === Seq(1001))
}
test("SPARK-24596 Non-cascading Cache Invalidation - uncache temporary view") {
@@ -1102,7 +1102,7 @@
.agg(avg("c1").as("v1"), sum("c2").as("v2"))
}
// First, checks if there is no column statistic in cached query
- val queryStats1 = query().cache.queryExecution.optimizedPlan.stats.attributeStats
+ val queryStats1 = query().cache().queryExecution.optimizedPlan.stats.attributeStats
assert(queryStats1.map(_._1.name).isEmpty)
val cacheManager = spark.sharedState.cacheManager
@@ -1596,8 +1596,8 @@
withTempDir { dir =>
val path1 = new File(dir, "t1").getCanonicalPath
val path2 = new File(dir, "t2").getCanonicalPath
- Seq(1).toDF.write.parquet(path1)
- Seq(1).toDF.write.parquet(path2)
+ Seq(1).toDF().write.parquet(path1)
+ Seq(1).toDF().write.parquet(path2)
val (tempViewStr, viewName) = if (ident.database.nonEmpty) {
("GLOBAL TEMPORARY VIEW", s"${ident.database.get}.${ident.table}")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
index 772eb9f..8c9ad21 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
@@ -161,7 +161,7 @@
}
test("star qualified by data frame object") {
- val df = testData.toDF
+ val df = testData.toDF()
val goldAnswer = df.collect().toSeq
checkAnswer(df.select(df("*")), goldAnswer)
@@ -260,7 +260,7 @@
test("isNull") {
checkAnswer(
- nullStrings.toDF.where($"s".isNull),
+ nullStrings.toDF().where($"s".isNull),
nullStrings.collect().toSeq.filter(r => r.getString(1) eq null))
checkAnswer(
@@ -270,7 +270,7 @@
test("isNotNull") {
checkAnswer(
- nullStrings.toDF.where($"s".isNotNull),
+ nullStrings.toDF().where($"s".isNotNull),
nullStrings.collect().toSeq.filter(r => r.getString(1) ne null))
checkAnswer(
@@ -539,41 +539,41 @@
withSQLConf(
SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> optThreshold.toString,
SQLConf.OPTIMIZER_INSET_SWITCH_THRESHOLD.key -> switchThreshold.toString) {
- checkAnswer(Seq(0).toDS.select($"value".isInCollection(Seq(null))), Seq(Row(null)))
+ checkAnswer(Seq(0).toDS().select($"value".isInCollection(Seq(null))), Seq(Row(null)))
checkAnswer(
- Seq(true).toDS.select($"value".isInCollection(Seq(true, false))),
+ Seq(true).toDS().select($"value".isInCollection(Seq(true, false))),
Seq(Row(true)))
checkAnswer(
- Seq(0.toByte, 1.toByte).toDS.select($"value".isInCollection(Seq(0.toByte, 2.toByte))),
+ Seq(0.toByte, 1.toByte).toDS().select($"value".isInCollection(Seq(0.toByte, 2.toByte))),
expected)
checkAnswer(
- Seq(0.toShort, 1.toShort).toDS
+ Seq(0.toShort, 1.toShort).toDS()
.select($"value".isInCollection(Seq(0.toShort, 2.toShort))),
expected)
- checkAnswer(Seq(0, 1).toDS.select($"value".isInCollection(Seq(0, 2))), expected)
- checkAnswer(Seq(0L, 1L).toDS.select($"value".isInCollection(Seq(0L, 2L))), expected)
- checkAnswer(Seq(0.0f, 1.0f).toDS
+ checkAnswer(Seq(0, 1).toDS().select($"value".isInCollection(Seq(0, 2))), expected)
+ checkAnswer(Seq(0L, 1L).toDS().select($"value".isInCollection(Seq(0L, 2L))), expected)
+ checkAnswer(Seq(0.0f, 1.0f).toDS()
.select($"value".isInCollection(Seq(0.0f, 2.0f))), expected)
- checkAnswer(Seq(0.0D, 1.0D).toDS
+ checkAnswer(Seq(0.0D, 1.0D).toDS()
.select($"value".isInCollection(Seq(0.0D, 2.0D))), expected)
checkAnswer(
- Seq(BigDecimal(0), BigDecimal(2)).toDS
+ Seq(BigDecimal(0), BigDecimal(2)).toDS()
.select($"value".isInCollection(Seq(BigDecimal(0), BigDecimal(1)))),
expected)
checkAnswer(
- Seq("abc", "def").toDS.select($"value".isInCollection(Seq("abc", "xyz"))),
+ Seq("abc", "def").toDS().select($"value".isInCollection(Seq("abc", "xyz"))),
expected)
checkAnswer(
- Seq(Date.valueOf("2020-04-29"), Date.valueOf("2020-05-01")).toDS
+ Seq(Date.valueOf("2020-04-29"), Date.valueOf("2020-05-01")).toDS()
.select($"value".isInCollection(
Seq(Date.valueOf("2020-04-29"), Date.valueOf("2020-04-30")))),
expected)
checkAnswer(
- Seq(new Timestamp(0), new Timestamp(2)).toDS
+ Seq(new Timestamp(0), new Timestamp(2)).toDS()
.select($"value".isInCollection(Seq(new Timestamp(0), new Timestamp(1)))),
expected)
checkAnswer(
- Seq(Array("a", "b"), Array("c", "d")).toDS
+ Seq(Array("a", "b"), Array("c", "d")).toDS()
.select($"value".isInCollection(Seq(Array("a", "b"), Array("x", "z")))),
expected)
}
@@ -3118,12 +3118,12 @@
test("SPARK-39093: divide period by integral expression") {
val df = Seq(((Period.ofMonths(10)), 2)).toDF("pd", "num")
checkAnswer(df.select($"pd" / ($"num" + 3)),
- Seq((Period.ofMonths(2))).toDF)
+ Seq((Period.ofMonths(2))).toDF())
}
test("SPARK-39093: divide duration by integral expression") {
val df = Seq(((Duration.ofDays(10)), 2)).toDF("dd", "num")
checkAnswer(df.select($"dd" / ($"num" + 3)),
- Seq((Duration.ofDays(2))).toDF)
+ Seq((Duration.ofDays(2))).toDF())
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
index 3b5f094..c40ecb8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
@@ -449,11 +449,11 @@
.selectExpr("from_csv(csv, 'a int, b int', map('mode', 'failfast')) as parsed")
val err1 = intercept[SparkException] {
- df.selectExpr("parsed.a").collect
+ df.selectExpr("parsed.a").collect()
}
val err2 = intercept[SparkException] {
- df.selectExpr("parsed.b").collect
+ df.selectExpr("parsed.b").collect()
}
assert(err1.getMessage.contains("Malformed records are detected in record parsing"))
@@ -475,7 +475,7 @@
}
test("SPARK-35998: Make from_csv/to_csv to handle year-month intervals properly") {
- val ymDF = Seq(Period.of(1, 2, 0)).toDF
+ val ymDF = Seq(Period.of(1, 2, 0)).toDF()
Seq(
(YearMonthIntervalType(), "INTERVAL '1-2' YEAR TO MONTH", Period.of(1, 2, 0)),
(YearMonthIntervalType(YEAR), "INTERVAL '1' YEAR", Period.of(1, 0, 0)),
@@ -502,7 +502,7 @@
}
test("SPARK-35999: Make from_csv/to_csv to handle day-time intervals properly") {
- val dtDF = Seq(Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)).toDF
+ val dtDF = Seq(Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)).toDF()
Seq(
(DayTimeIntervalType(), "INTERVAL '1 02:03:04' DAY TO SECOND",
Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)),
@@ -547,7 +547,7 @@
test("SPARK-36490: Make from_csv/to_csv to handle timestamp_ntz type properly") {
val localDT = LocalDateTime.parse("2021-08-12T15:16:23")
- val df = Seq(localDT).toDF
+ val df = Seq(localDT).toDF()
val toCsvDF = df.select(to_csv(struct($"value")) as "csv")
checkAnswer(toCsvDF, Row("2021-08-12T15:16:23.000"))
val fromCsvDF = toCsvDF
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
index 80862ee..ccfdddd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
@@ -158,7 +158,7 @@
Fact(20151123, 18, 36, "room2", 25.6))).toDF()
val cube0 = df0.cube("date", "hour", "minute", "room_name").agg(Map("temp" -> "avg"))
- assert(cube0.where("date IS NULL").count > 0)
+ assert(cube0.where("date IS NULL").count() > 0)
}
test("grouping and grouping_id") {
@@ -960,7 +960,7 @@
.select($"x", map($"x", $"y").as("y"))
.createOrReplaceTempView("tempView")
val error = intercept[AnalysisException] {
- sql("SELECT max_by(x, y) FROM tempView").show
+ sql("SELECT max_by(x, y) FROM tempView").show()
}
checkError(
exception = error,
@@ -1030,7 +1030,7 @@
.select($"x", map($"x", $"y").as("y"))
.createOrReplaceTempView("tempView")
val error = intercept[AnalysisException] {
- sql("SELECT min_by(x, y) FROM tempView").show
+ sql("SELECT min_by(x, y) FROM tempView").show()
}
checkError(
exception = error,
@@ -1242,7 +1242,7 @@
val df = Seq(
A(None),
A(Some(B(None))),
- A(Some(B(Some(1.0))))).toDF
+ A(Some(B(Some(1.0))))).toDF()
val groupBy = df.groupBy("b").agg(count("*"))
checkAnswer(groupBy, Row(null, 1) :: Row(Row(null), 1) :: Row(Row(1.0), 1) :: Nil)
}
@@ -1613,7 +1613,7 @@
test("SPARK-38185: Fix data incorrect if aggregate function is empty") {
val emptyAgg = Map.empty[String, String]
- assert(spark.range(2).where("id > 2").agg(emptyAgg).limit(1).count == 1)
+ assert(spark.range(2).where("id > 2").agg(emptyAgg).limit(1).count() == 1)
}
test("SPARK-38221: group by stream of complex expressions should not fail") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
index 4f25642..04c2b5a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
@@ -66,7 +66,7 @@
test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") {
val ds100_5 = Seq(S100_5()).toDS()
- ds100_5.rdd.count
+ ds100_5.rdd.count()
}
test("SPARK-29503 nest unsafe struct inside safe array") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
index 4020688..7891965 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
@@ -176,7 +176,7 @@
val df5 = Seq((Seq("a", null), Seq(1, 2))).toDF("k", "v")
val e1 = intercept[SparkException] {
- df5.select(map_from_arrays($"k", $"v")).collect
+ df5.select(map_from_arrays($"k", $"v")).collect()
}
assert(e1.getCause.isInstanceOf[SparkRuntimeException])
checkError(
@@ -187,7 +187,7 @@
val df6 = Seq((Seq(1, 2), Seq("a"))).toDF("k", "v")
val msg2 = intercept[Exception] {
- df6.select(map_from_arrays($"k", $"v")).collect
+ df6.select(map_from_arrays($"k", $"v")).collect()
}.getMessage
assert(msg2.contains("The key array and value array of MapData must have the same length"))
}
@@ -224,7 +224,7 @@
StructField("col1", IntegerType, nullable = false),
StructField("b", StringType)
))
- assert(result.first.schema(0).dataType === expectedType)
+ assert(result.first().schema(0).dataType === expectedType)
checkAnswer(result, Row(Row(2, "str")))
}
@@ -237,7 +237,7 @@
StructField("col2", DoubleType, nullable = false)
))
- assert(result.first.schema(0).dataType === expectedType)
+ assert(result.first().schema(0).dataType === expectedType)
checkAnswer(result, Seq(Row(Row(2, 5.0)), Row(Row(4, 5.0))))
}
@@ -250,7 +250,7 @@
StructField("col2", DoubleType, nullable = false)
))
- assert(result.first.schema(0).dataType === expectedType)
+ assert(result.first().schema(0).dataType === expectedType)
checkAnswer(result, Seq(Row(Row("v", 5.0)), Row(Row("v", 5.0))))
}
@@ -401,7 +401,7 @@
val encryptedEcb = "9J3iZbIxnmaG+OIA9Amd+A=="
val encryptedGcm = "y5la3muiuxN2suj6VsYXB+0XUFjtrUD0/zv5eDafsA3U"
val encryptedCbc = "+MgyzJxhusYVGWCljk7fhhl6C6oUqWmtdqoaG93KvhY="
- val df1 = Seq("Spark").toDF
+ val df1 = Seq("Spark").toDF()
// Successful decryption of fixed values
Seq(
@@ -426,7 +426,7 @@
val encryptedGcm = "AAAAAAAAAAAAAAAAQiYi+sRNYDAOTjdSEcYBFsAWPL1f"
val cbcIv = "00000000000000000000000000000000"
val encryptedCbc = "AAAAAAAAAAAAAAAAAAAAAPSd4mWyMZ5mhvjiAPQJnfg="
- val df1 = Seq("Spark").toDF
+ val df1 = Seq("Spark").toDF()
Seq(
(key32, encryptedGcm, "GCM", gcmIv),
(key32, encryptedCbc, "CBC", cbcIv)).foreach {
@@ -451,7 +451,7 @@
val gcmIv = "000000000000000000000000"
val aad = "This is an AAD mixed into the input"
val encryptedGcm = "AAAAAAAAAAAAAAAAQiYi+sTLm7KD9UcZ2nlRdYDe/PX4"
- val df1 = Seq("Spark").toDF
+ val df1 = Seq("Spark").toDF()
Seq(
(key32, encryptedGcm, "GCM", gcmIv, aad)).foreach {
case (key, ciphertext, mode, iv, aad) =>
@@ -481,7 +481,7 @@
val encryptedEmptyText24 = "9RDK70sHNzqAFRcpfGM5gQ=="
val encryptedEmptyText32 = "j9IDsCvlYXtcVJUf4FAjQQ=="
- val df1 = Seq("Spark", "").toDF
+ val df1 = Seq("Spark", "").toDF()
// Successful encryption
Seq(
@@ -3232,7 +3232,7 @@
).toDF("a", "b")
val result10 = df10.select(array_except($"a", $"b"))
val expectedType10 = ArrayType(IntegerType, containsNull = true)
- assert(result10.first.schema(0).dataType === expectedType10)
+ assert(result10.first().schema(0).dataType === expectedType10)
}
test("array_intersect functions") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala
index a454d91..8850e4d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala
@@ -53,13 +53,13 @@
}
test("SPARK-19959: df[java.lang.Long].collect includes null throws NullPointerException") {
- checkAnswer(sparkContext.parallelize(Seq[java.lang.Integer](0, null, 2), 1).toDF,
+ checkAnswer(sparkContext.parallelize(Seq[java.lang.Integer](0, null, 2), 1).toDF(),
Seq(Row(0), Row(null), Row(2)))
- checkAnswer(sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF,
+ checkAnswer(sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF(),
Seq(Row(0L), Row(null), Row(2L)))
- checkAnswer(sparkContext.parallelize(Seq[java.lang.Float](0.0F, null, 2.0F), 1).toDF,
+ checkAnswer(sparkContext.parallelize(Seq[java.lang.Float](0.0F, null, 2.0F), 1).toDF(),
Seq(Row(0.0F), Row(null), Row(2.0F)))
- checkAnswer(sparkContext.parallelize(Seq[java.lang.Double](0.0D, null, 2.0D), 1).toDF,
+ checkAnswer(sparkContext.parallelize(Seq[java.lang.Double](0.0D, null, 2.0D), 1).toDF(),
Seq(Row(0.0D), Row(null), Row(2.0D)))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
index a923d1f..4c6f34d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
@@ -285,7 +285,7 @@
test("process outer join results using the non-nullable columns in the join input") {
// Filter data using a non-nullable column from a right table
val df1 = Seq((0, 0), (1, 0), (2, 0), (3, 0), (4, 0)).toDF("id", "count")
- val df2 = Seq(Tuple1(0), Tuple1(1)).toDF("id").groupBy("id").count
+ val df2 = Seq(Tuple1(0), Tuple1(1)).toDF("id").groupBy("id").count()
checkAnswer(
df1.join(df2, df1("id") === df2("id"), "left_outer").filter(df2("count").isNull),
Row(2, 0, null, null) ::
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
index ea0e9a3..ab900e2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
@@ -320,7 +320,7 @@
val df = createDFWithNestedColumns
// Rows with the specified nested columns whose null values are dropped.
- assert(df.count == 3)
+ assert(df.count() == 3)
checkAnswer(
df.na.drop("any", Seq("c1.c1-1")),
Seq(Row(Row("b1", "b2"))))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
index eafd454..237915f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
@@ -332,7 +332,7 @@
(2, Seq("a", "x")),
(3, Seq.empty[String]),
(3, Seq("a", "x"))).toDF("x", "s")
- val expected = Seq((3, 1, 1), (2, 1, 1)).toDF
+ val expected = Seq((3, 1, 1), (2, 1, 1)).toDF()
val actual = df.groupBy("x").pivot("s").count()
checkAnswer(actual, expected)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala
index 917f80e..441b276 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala
@@ -35,77 +35,77 @@
test("SPARK-7150 range api") {
// numSlice is greater than length
val res1 = spark.range(0, 10, 1, 15).select("id")
- assert(res1.count == 10)
+ assert(res1.count() == 10)
assert(res1.agg(sum("id")).as("sumid").collect() === Seq(Row(45)))
val res2 = spark.range(3, 15, 3, 2).select("id")
- assert(res2.count == 4)
+ assert(res2.count() == 4)
assert(res2.agg(sum("id")).as("sumid").collect() === Seq(Row(30)))
val res3 = spark.range(1, -2).select("id")
- assert(res3.count == 0)
+ assert(res3.count() == 0)
// start is positive, end is negative, step is negative
val res4 = spark.range(1, -2, -2, 6).select("id")
- assert(res4.count == 2)
+ assert(res4.count() == 2)
assert(res4.agg(sum("id")).as("sumid").collect() === Seq(Row(0)))
// start, end, step are negative
val res5 = spark.range(-3, -8, -2, 1).select("id")
- assert(res5.count == 3)
+ assert(res5.count() == 3)
assert(res5.agg(sum("id")).as("sumid").collect() === Seq(Row(-15)))
// start, end are negative, step is positive
val res6 = spark.range(-8, -4, 2, 1).select("id")
- assert(res6.count == 2)
+ assert(res6.count() == 2)
assert(res6.agg(sum("id")).as("sumid").collect() === Seq(Row(-14)))
val res7 = spark.range(-10, -9, -20, 1).select("id")
- assert(res7.count == 0)
+ assert(res7.count() == 0)
if (!conf.ansiEnabled) {
val res8 = spark.range(Long.MinValue, Long.MaxValue, Long.MaxValue, 100).select("id")
- assert(res8.count == 3)
+ assert(res8.count() == 3)
assert(res8.agg(sum("id")).as("sumid").collect() === Seq(Row(-3)))
val res9 = spark.range(Long.MaxValue, Long.MinValue, Long.MinValue, 100).select("id")
- assert(res9.count == 2)
+ assert(res9.count() == 2)
assert(res9.agg(sum("id")).as("sumid").collect() === Seq(Row(Long.MaxValue - 1)))
}
// only end provided as argument
val res10 = spark.range(10).select("id")
- assert(res10.count == 10)
+ assert(res10.count() == 10)
assert(res10.agg(sum("id")).as("sumid").collect() === Seq(Row(45)))
val res11 = spark.range(-1).select("id")
- assert(res11.count == 0)
+ assert(res11.count() == 0)
// using the default slice number
val res12 = spark.range(3, 15, 3).select("id")
- assert(res12.count == 4)
+ assert(res12.count() == 4)
assert(res12.agg(sum("id")).as("sumid").collect() === Seq(Row(30)))
// difference between range start and end does not fit in a 64-bit integer
val n = 9L * 1000 * 1000 * 1000 * 1000 * 1000 * 1000
val res13 = spark.range(-n, n, n / 9).select("id")
- assert(res13.count == 18)
+ assert(res13.count() == 18)
// range with non aggregation operation
- val res14 = spark.range(0, 100, 2).toDF.filter("50 <= id")
- val len14 = res14.collect.length
+ val res14 = spark.range(0, 100, 2).toDF().filter("50 <= id")
+ val len14 = res14.collect().length
assert(len14 == 25)
- val res15 = spark.range(100, -100, -2).toDF.filter("id <= 0")
- val len15 = res15.collect.length
+ val res15 = spark.range(100, -100, -2).toDF().filter("id <= 0")
+ val len15 = res15.collect().length
assert(len15 == 50)
- val res16 = spark.range(-1500, 1500, 3).toDF.filter("0 <= id")
- val len16 = res16.collect.length
+ val res16 = spark.range(-1500, 1500, 3).toDF().filter("0 <= id")
+ val len16 = res16.collect().length
assert(len16 == 500)
- val res17 = spark.range(10, 0, -1, 1).toDF.sortWithinPartitions("id")
- assert(res17.collect === (1 to 10).map(i => Row(i)).toArray)
+ val res17 = spark.range(10, 0, -1, 1).toDF().sortWithinPartitions("id")
+ assert(res17.collect() === (1 to 10).map(i => Row(i)).toArray)
}
testWithWholeStageCodegenOnAndOff("Range with randomized parameters") { codegenEnabled =>
@@ -184,7 +184,7 @@
"inconsistent with SparkContext.range()") { _ =>
val start = java.lang.Long.MAX_VALUE - 3
val end = java.lang.Long.MIN_VALUE + 2
- assert(spark.range(start, end, 1).collect.length == 0)
- assert(spark.range(start, start, 1).collect.length == 0)
+ assert(spark.range(start, end, 1).collect().length == 0)
+ assert(spark.range(start, start, 1).collect().length == 0)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala
index b830039..88ef593 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala
@@ -357,7 +357,7 @@
assertAmbiguousSelfJoin(df2.join(df1, df1("key1") === df2("key2")))
// Test for SerializeFromObject
- val df3 = spark.sparkContext.parallelize(1 to 10).map(x => (x, x)).toDF
+ val df3 = spark.sparkContext.parallelize(1 to 10).map(x => (x, x)).toDF()
val df4 = df3.filter($"_1" <=> 0)
assertAmbiguousSelfJoin(df3.join(df4, df3("_1") === df4("_2")))
assertAmbiguousSelfJoin(df4.join(df3, df3("_1") === df4("_2")))
@@ -411,7 +411,7 @@
// Test for Generate
// Ensure that the root of the plan is Generate
val df15 = Seq((1, Seq(1, 2, 3))).toDF("a", "intList").select($"a", explode($"intList"))
- .queryExecution.optimizedPlan.find(_.isInstanceOf[Generate]).get.toDF
+ .queryExecution.optimizedPlan.find(_.isInstanceOf[Generate]).get.toDF()
val df16 = df15.filter($"a" > 0)
assertAmbiguousSelfJoin(df15.join(df16, df15("a") === df16("col")))
assertAmbiguousSelfJoin(df16.join(df15, df15("a") === df16("col")))
@@ -424,7 +424,7 @@
Seq(
AttributeReference("x", IntegerType)(),
AttributeReference("y", IntegerType)()),
- df1.queryExecution.logical).toDF
+ df1.queryExecution.logical).toDF()
val df18 = df17.filter($"x" > 0)
assertAmbiguousSelfJoin(df17.join(df18, df17("x") === df18("y")))
assertAmbiguousSelfJoin(df18.join(df17, df17("x") === df18("y")))
@@ -436,7 +436,7 @@
Seq(Alias(dfWithTS("time").expr, "ts")()),
Seq(dfWithTS("a").expr),
Seq(SortOrder(dfWithTS("a").expr, Ascending)),
- dfWithTS.queryExecution.logical).toDF
+ dfWithTS.queryExecution.logical).toDF()
val df20 = df19.filter($"a" > 0)
assertAmbiguousSelfJoin(df19.join(df20, df19("a") === df20("b")))
assertAmbiguousSelfJoin(df20.join(df19, df19("a") === df20("b")))
@@ -462,7 +462,7 @@
AttributeReference("x", IntegerType)(),
AttributeReference("y", IntegerType)()),
df1.queryExecution.logical,
- ioSchema).toDF
+ ioSchema).toDF()
val df22 = df21.filter($"x" > 0)
assertAmbiguousSelfJoin(df21.join(df22, df21("x") === df22("y")))
assertAmbiguousSelfJoin(df22.join(df21, df21("x") === df22("y")))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala
index 747f43f..ab8aab0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala
@@ -633,7 +633,7 @@
(1, 1)
)).toDF("a", "b").withColumn("c", newCol)
- val df2 = df1.union(df1).withColumn("d", spark_partition_id).filter(filter)
+ val df2 = df1.union(df1).withColumn("d", spark_partition_id()).filter(filter)
checkAnswer(df2, result)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index c72bc91..b7450e5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -989,7 +989,7 @@
.parallelize(Seq(StringWrapper("a"), StringWrapper("b"), StringWrapper("c")))
.toDF()
val filtered = df.where("s = \"a\"")
- checkAnswer(filtered, spark.sparkContext.parallelize(Seq(StringWrapper("a"))).toDF)
+ checkAnswer(filtered, spark.sparkContext.parallelize(Seq(StringWrapper("a"))).toDF())
}
test("SPARK-20384: Tuple2 of value class filter") {
@@ -1000,7 +1000,7 @@
.toDF()
val filtered = df.where("_2.s = \"a2\"")
checkAnswer(filtered,
- spark.sparkContext.parallelize(Seq((StringWrapper("a1"), StringWrapper("a2")))).toDF)
+ spark.sparkContext.parallelize(Seq((StringWrapper("a1"), StringWrapper("a2")))).toDF())
}
test("SPARK-20384: Tuple3 of value class filter") {
@@ -1012,26 +1012,26 @@
val filtered = df.where("_3.s = \"a3\"")
checkAnswer(filtered,
spark.sparkContext.parallelize(
- Seq((StringWrapper("a1"), StringWrapper("a2"), StringWrapper("a3")))).toDF)
+ Seq((StringWrapper("a1"), StringWrapper("a2"), StringWrapper("a3")))).toDF())
}
test("SPARK-20384: Array value class filter") {
val ab = ArrayStringWrapper(Seq(StringWrapper("a"), StringWrapper("b")))
val cd = ArrayStringWrapper(Seq(StringWrapper("c"), StringWrapper("d")))
- val df = spark.sparkContext.parallelize(Seq(ab, cd)).toDF
+ val df = spark.sparkContext.parallelize(Seq(ab, cd)).toDF()
val filtered = df.where(array_contains(col("wrappers.s"), "b"))
- checkAnswer(filtered, spark.sparkContext.parallelize(Seq(ab)).toDF)
+ checkAnswer(filtered, spark.sparkContext.parallelize(Seq(ab)).toDF())
}
test("SPARK-20384: Nested value class filter") {
val a = ContainerStringWrapper(StringWrapper("a"))
val b = ContainerStringWrapper(StringWrapper("b"))
- val df = spark.sparkContext.parallelize(Seq(a, b)).toDF
+ val df = spark.sparkContext.parallelize(Seq(a, b)).toDF()
// flat value class, `s` field is not in schema
val filtered = df.where("wrapper = \"a\"")
- checkAnswer(filtered, spark.sparkContext.parallelize(Seq(a)).toDF)
+ checkAnswer(filtered, spark.sparkContext.parallelize(Seq(a)).toDF())
}
private lazy val person2: DataFrame = Seq(
@@ -2474,7 +2474,7 @@
val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(Seq.range(0, size))))
val schemas = List.range(0, size).map(a => StructField("name" + a, LongType, true))
val df = spark.createDataFrame(rdd, StructType(schemas))
- assert(df.persist.take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100)
+ assert(df.persist().take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100)
}
test("SPARK-17409: Do Not Optimize Query in CTAS (Data source tables) More Than Once") {
@@ -2498,7 +2498,7 @@
test("copy results for sampling with replacement") {
val df = Seq((1, 0), (2, 0), (3, 0)).toDF("a", "b")
val sampleDf = df.sample(true, 2.00)
- val d = sampleDf.withColumn("c", monotonically_increasing_id).select($"c").collect
+ val d = sampleDf.withColumn("c", monotonically_increasing_id()).select($"c").collect()
assert(d.size == d.distinct.size)
}
@@ -2601,7 +2601,7 @@
df1
.join(df2, df1("x") === df2("x1"), "left_outer")
.filter($"x1".isNotNull || !$"y".isin("a!"))
- .count
+ .count()
}
// The fix of SPARK-21720 avoid an exception regarding JVM code size limit
@@ -2862,7 +2862,7 @@
Console.withOut(captured) {
df.explain(extended = true)
}
- checkAnswer(df, spark.range(10).toDF)
+ checkAnswer(df, spark.range(10).toDF())
val output = captured.toString
assert(output.contains(
"""== Parsed Logical Plan ==
@@ -2905,7 +2905,7 @@
data1.zip(data2).map { p =>
p._1.getInt(2) + p._2.getInt(2)
}
- }.toDF
+ }.toDF()
checkAnswer(df3.sort("value"), Row(7) :: Row(9) :: Nil)
@@ -2932,7 +2932,7 @@
data1.zip(data2).map { p =>
p._1.getInt(2) + p._2.getInt(2)
}
- }.toDF
+ }.toDF()
checkAnswer(df3.sort("value"), Row(7) :: Row(9) :: Nil)
}
@@ -3459,7 +3459,7 @@
}
))
- assert(res.collect.length == 2)
+ assert(res.collect().length == 2)
}
test("SPARK-38285: Fix ClassCastException: GenericArrayData cannot be cast to InternalRow") {
@@ -3524,7 +3524,7 @@
}
// UNION (distinct)
- val df4 = df3.distinct
+ val df4 = df3.distinct()
checkAnswer(df4, rows.distinct)
val t4 = sqlQuery(cols1, cols2, true)
@@ -3546,7 +3546,7 @@
}
test("SPARK-39612: exceptAll with following count should work") {
- val d1 = Seq("a").toDF
+ val d1 = Seq("a").toDF()
assert(d1.exceptAll(d1).count() === 0)
}
@@ -3641,7 +3641,7 @@
val df = spark.sparkContext.parallelize(1 to 5).toDF("x")
val v1 = (rand() * 10000).cast(IntegerType)
val v2 = to_csv(struct(v1.as("a"))) // to_csv is CodegenFallback
- df.select(v1, v1, v2, v2).collect.foreach { row =>
+ df.select(v1, v1, v2, v2).collect().foreach { row =>
assert(row.getInt(0) == row.getInt(1))
assert(row.getInt(0).toString == row.getString(2))
assert(row.getInt(0).toString == row.getString(3))
@@ -3656,7 +3656,7 @@
val r3 = random()
val r4 = uuid()
val r5 = shuffle(col("a"))
- df.select(r, r, r2, r2, r3, r3, r4, r4, r5, r5).collect.foreach { row =>
+ df.select(r, r, r2, r2, r3, r3, r4, r4, r5, r5).collect().foreach { row =>
(0 until 5).foreach(i => assert(row.get(i * 2) === row.get(i * 2 + 1)))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala
index a9f7d5b..e5937ce 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala
@@ -84,19 +84,19 @@
}
val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data)))
val df = spark.createDataFrame(rdd, StructType(schemas))
- val row = df.persist.take(1).apply(0)
+ val row = df.persist().take(1).apply(0)
checkAnswer(df, row)
}
test("access cache multiple times") {
- val df0 = sparkContext.parallelize(Seq(1, 2, 3), 1).toDF("x").cache
- df0.count
+ val df0 = sparkContext.parallelize(Seq(1, 2, 3), 1).toDF("x").cache()
+ df0.count()
val df1 = df0.filter("x > 1")
checkAnswer(df1, Seq(Row(2), Row(3)))
val df2 = df0.filter("x > 2")
checkAnswer(df2, Row(3))
- val df10 = sparkContext.parallelize(Seq(3, 4, 5, 6), 1).toDF("x").cache
+ val df10 = sparkContext.parallelize(Seq(3, 4, 5, 6), 1).toDF("x").cache()
for (_ <- 0 to 2) {
val df11 = df10.filter("x > 5")
checkAnswer(df11, Row(6))
@@ -105,8 +105,8 @@
test("access only some column of the all of columns") {
val df = spark.range(1, 10).map(i => (i, (i + 1).toDouble)).toDF("l", "d")
- df.cache
- df.count
- assert(df.filter("d < 3").count == 1)
+ df.cache()
+ df.count()
+ assert(df.filter("d < 3").count() == 1)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala
index 076332f..dcd16ba 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala
@@ -252,7 +252,7 @@
}
override def getSparkSession: SparkSession = {
- SparkSession.builder
+ SparkSession.builder()
.master("local[*]")
.appName("Dataset benchmark")
.getOrCreate()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 10a28ca..51fa3cd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -1018,7 +1018,7 @@
val observed_df = spark.range(100).observe(
namedObservation, percentile_approx($"id", lit(0.5), lit(100)).as("percentile_approx_val"))
- observed_df.foreach(r => f)
+ observed_df.foreach(r => f())
val expected = Map("percentile_approx_val" -> 49)
assert(namedObservation.get === expected)
@@ -1089,9 +1089,9 @@
).toDF("id", "stringData")
val sampleDF = df.sample(false, 0.7, 50)
// After sampling, sampleDF doesn't contain id=a.
- assert(!sampleDF.select("id").as[Int].collect.contains(a))
+ assert(!sampleDF.select("id").as[Int].collect().contains(a))
// simpleUdf should not encounter id=a.
- checkAnswer(sampleDF.select(simpleUdf($"id")), List.fill(sampleDF.count.toInt)(Row(a)))
+ checkAnswer(sampleDF.select(simpleUdf($"id")), List.fill(sampleDF.count().toInt)(Row(a)))
}
test("SPARK-11436: we should rebind right encoder when join 2 datasets") {
@@ -1166,7 +1166,7 @@
}
test("SPARK-14696: implicit encoders for boxed types") {
- assert(spark.range(1).map { i => i : java.lang.Long }.head == 0L)
+ assert(spark.range(1).map { i => i : java.lang.Long }.head() == 0L)
}
test("SPARK-11894: Incorrect results are returned when using null") {
@@ -1336,11 +1336,11 @@
test("dataset.rdd with generic case class") {
val ds = Seq(Generic(1, 1.0), Generic(2, 2.0)).toDS()
val ds2 = ds.map(g => Generic(g.id, g.value))
- assert(ds.rdd.map(r => r.id).count === 2)
- assert(ds2.rdd.map(r => r.id).count === 2)
+ assert(ds.rdd.map(r => r.id).count() === 2)
+ assert(ds2.rdd.map(r => r.id).count() === 2)
val ds3 = ds.map(g => java.lang.Long.valueOf(g.id))
- assert(ds3.rdd.map(r => r).count === 2)
+ assert(ds3.rdd.map(r => r).count() === 2)
}
test("runtime null check for RowEncoder") {
@@ -1614,7 +1614,7 @@
Route("b", "a", 1),
Route("b", "a", 5),
Route("b", "c", 6))
- val ds = sparkContext.parallelize(data).toDF.as[Route]
+ val ds = sparkContext.parallelize(data).toDF().as[Route]
val grouped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r)))
.groupByKey(r => (r.src, r.dest))
@@ -1649,15 +1649,15 @@
}
test("SPARK-18284: Serializer should have correct nullable value") {
- val df1 = Seq(1, 2, 3, 4).toDF
+ val df1 = Seq(1, 2, 3, 4).toDF()
assert(df1.schema(0).nullable == false)
- val df2 = Seq(Integer.valueOf(1), Integer.valueOf(2)).toDF
+ val df2 = Seq(Integer.valueOf(1), Integer.valueOf(2)).toDF()
assert(df2.schema(0).nullable)
- val df3 = Seq(Seq(1, 2), Seq(3, 4)).toDF
+ val df3 = Seq(Seq(1, 2), Seq(3, 4)).toDF()
assert(df3.schema(0).nullable)
assert(df3.schema(0).dataType.asInstanceOf[ArrayType].containsNull == false)
- val df4 = Seq(Seq("a", "b"), Seq("c", "d")).toDF
+ val df4 = Seq(Seq("a", "b"), Seq("c", "d")).toDF()
assert(df4.schema(0).nullable)
assert(df4.schema(0).dataType.asInstanceOf[ArrayType].containsNull)
@@ -1686,7 +1686,7 @@
assert(df10.schema(0).dataType.asInstanceOf[MapType].valueContainsNull)
val df11 = Seq(TestDataPoint(1, 2.2, "a", null),
- TestDataPoint(3, 4.4, "null", (TestDataPoint2(33, "b")))).toDF
+ TestDataPoint(3, 4.4, "null", (TestDataPoint2(33, "b")))).toDF()
assert(df11.schema(0).nullable == false)
assert(df11.schema(1).nullable == false)
assert(df11.schema(2).nullable)
@@ -1802,11 +1802,11 @@
val arrayLong = Array(1.toLong, 2.toLong, 3.toLong)
val arrayDouble = Array(1.1, 2.2, 3.3)
val arrayString = Array("a", "b", "c")
- val dsByte = sparkContext.parallelize(Seq(arrayByte), 1).toDS.map(e => e)
- val dsInt = sparkContext.parallelize(Seq(arrayInt), 1).toDS.map(e => e)
- val dsLong = sparkContext.parallelize(Seq(arrayLong), 1).toDS.map(e => e)
- val dsDouble = sparkContext.parallelize(Seq(arrayDouble), 1).toDS.map(e => e)
- val dsString = sparkContext.parallelize(Seq(arrayString), 1).toDS.map(e => e)
+ val dsByte = sparkContext.parallelize(Seq(arrayByte), 1).toDS().map(e => e)
+ val dsInt = sparkContext.parallelize(Seq(arrayInt), 1).toDS().map(e => e)
+ val dsLong = sparkContext.parallelize(Seq(arrayLong), 1).toDS().map(e => e)
+ val dsDouble = sparkContext.parallelize(Seq(arrayDouble), 1).toDS().map(e => e)
+ val dsString = sparkContext.parallelize(Seq(arrayString), 1).toDS().map(e => e)
checkDataset(dsByte, arrayByte)
checkDataset(dsInt, arrayInt)
checkDataset(dsLong, arrayLong)
@@ -1826,41 +1826,41 @@
test("SPARK-18717: code generation works for both scala.collection.Map" +
" and scala.collection.immutable.Map") {
- val ds = Seq(WithImmutableMap("hi", Map(42L -> "foo"))).toDS
+ val ds = Seq(WithImmutableMap("hi", Map(42L -> "foo"))).toDS()
checkDataset(ds.map(t => t), WithImmutableMap("hi", Map(42L -> "foo")))
- val ds2 = Seq(WithMap("hi", Map(42L -> "foo"))).toDS
+ val ds2 = Seq(WithMap("hi", Map(42L -> "foo"))).toDS()
checkDataset(ds2.map(t => t), WithMap("hi", Map(42L -> "foo")))
}
test("SPARK-18746: add implicit encoder for BigDecimal, date, timestamp") {
// For this implicit encoder, 18 is the default scale
- assert(spark.range(1).map { x => new java.math.BigDecimal(1) }.head ==
+ assert(spark.range(1).map { x => new java.math.BigDecimal(1) }.head() ==
new java.math.BigDecimal(1).setScale(18))
- assert(spark.range(1).map { x => scala.math.BigDecimal(1, 18) }.head ==
+ assert(spark.range(1).map { x => scala.math.BigDecimal(1, 18) }.head() ==
scala.math.BigDecimal(1, 18))
- assert(spark.range(1).map { x => java.sql.Date.valueOf("2016-12-12") }.head ==
+ assert(spark.range(1).map { x => java.sql.Date.valueOf("2016-12-12") }.head() ==
java.sql.Date.valueOf("2016-12-12"))
- assert(spark.range(1).map { x => new java.sql.Timestamp(100000) }.head ==
+ assert(spark.range(1).map { x => new java.sql.Timestamp(100000) }.head() ==
new java.sql.Timestamp(100000))
}
test("SPARK-19896: cannot have circular references in case class") {
val errMsg1 = intercept[UnsupportedOperationException] {
- Seq(CircularReferenceClassA(null)).toDS
+ Seq(CircularReferenceClassA(null)).toDS()
}
assert(errMsg1.getMessage.startsWith("cannot have circular references in class, but got the " +
"circular reference of class"))
val errMsg2 = intercept[UnsupportedOperationException] {
- Seq(CircularReferenceClassC(null)).toDS
+ Seq(CircularReferenceClassC(null)).toDS()
}
assert(errMsg2.getMessage.startsWith("cannot have circular references in class, but got the " +
"circular reference of class"))
val errMsg3 = intercept[UnsupportedOperationException] {
- Seq(CircularReferenceClassD(null)).toDS
+ Seq(CircularReferenceClassD(null)).toDS()
}
assert(errMsg3.getMessage.startsWith("cannot have circular references in class, but got the " +
"circular reference of class"))
@@ -2035,12 +2035,12 @@
test("SPARK-24569: Option of primitive types are mistakenly mapped to struct type") {
withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
- val a = Seq(Some(1)).toDS
- val b = Seq(Some(1.2)).toDS
- val expected = Seq((Some(1), Some(1.2))).toDS
+ val a = Seq(Some(1)).toDS()
+ val b = Seq(Some(1.2)).toDS()
+ val expected = Seq((Some(1), Some(1.2))).toDS()
val joined = a.joinWith(b, lit(true))
assert(joined.schema == expected.schema)
- checkDataset(joined, expected.collect: _*)
+ checkDataset(joined, expected.collect(): _*)
}
}
@@ -2186,7 +2186,7 @@
test("SPARK-8288: class with only a companion object constructor") {
val data = Seq(ScroogeLikeExample(1), ScroogeLikeExample(2))
- val ds = data.toDS
+ val ds = data.toDS()
checkDataset(ds, data: _*)
checkAnswer(ds.select("x"), Seq(Row(1), Row(2)))
}
@@ -2227,10 +2227,10 @@
test("implicit encoder for LocalDate and Instant") {
val localDate = java.time.LocalDate.of(2019, 3, 30)
- assert(spark.range(1).map { _ => localDate }.head === localDate)
+ assert(spark.range(1).map { _ => localDate }.head() === localDate)
val instant = java.time.Instant.parse("2019-03-30T09:54:00Z")
- assert(spark.range(1).map { _ => instant }.head === instant)
+ assert(spark.range(1).map { _ => instant }.head() === instant)
}
val dotColumnTestModes = Table(
@@ -2241,7 +2241,7 @@
test("SPARK-25153: Improve error messages for columns with dots/periods") {
forAll(dotColumnTestModes) { (caseSensitive, colName) =>
- val ds = Seq(SpecialCharClass("1", "2")).toDS
+ val ds = Seq(SpecialCharClass("1", "2")).toDS()
withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
val colName = if (caseSensitive == "true") "`Field`.`1`" else "`field`.`1`"
checkError(
@@ -2257,7 +2257,7 @@
test("SPARK-39783: Fix error messages for columns with dots/periods") {
forAll(dotColumnTestModes) { (caseSensitive, colName) =>
- val ds = Seq(SpecialCharClass("1", "2")).toDS
+ val ds = Seq(SpecialCharClass("1", "2")).toDS()
withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
checkError(
exception = intercept[AnalysisException] {
@@ -2352,9 +2352,9 @@
assert(df1.sameSemantics(df3) === false)
assert(df3.sameSemantics(df4) === true)
- assert(df1.semanticHash === df2.semanticHash)
- assert(df1.semanticHash !== df3.semanticHash)
- assert(df3.semanticHash === df4.semanticHash)
+ assert(df1.semanticHash() === df2.semanticHash())
+ assert(df1.semanticHash() !== df3.semanticHash())
+ assert(df3.semanticHash() === df4.semanticHash())
}
test("SPARK-31854: Invoke in MapElementsExec should not propagate null") {
@@ -2428,12 +2428,12 @@
test("SPARK-34605: implicit encoder for java.time.Duration") {
val duration = java.time.Duration.ofMinutes(10)
- assert(spark.range(1).map { _ => duration }.head === duration)
+ assert(spark.range(1).map { _ => duration }.head() === duration)
}
test("SPARK-34615: implicit encoder for java.time.Period") {
val period = java.time.Period.ofYears(9999).withMonths(11)
- assert(spark.range(1).map { _ => period }.head === period)
+ assert(spark.range(1).map { _ => period }.head() === period)
}
test("SPARK-35652: joinWith on two table generated from same one performing a cartesian join," +
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
index 555679e..50dcb9d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
@@ -1658,7 +1658,7 @@
test("no partition pruning when the build side is a stream") {
withTable("fact") {
val input = MemoryStream[Int]
- val stream = input.toDF.select($"value" as "one", ($"value" * 3) as "code")
+ val stream = input.toDF().select($"value" as "one", ($"value" * 3) as "code")
spark.range(100).select(
$"id",
($"id" + 1).as("one"),
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala
index a206e97..8b5ffe5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala
@@ -423,7 +423,7 @@
}
test("Dataset.toExplainString has mode as string") {
- val df = spark.range(10).toDF
+ val df = spark.range(10).toDF()
def assertExplainOutput(mode: ExplainMode): Unit = {
assert(df.queryExecution.explainString(mode).replaceAll("#\\d+", "#x").trim ===
getNormalizedExplain(df, mode).trim)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
index 9327548..fc68f27 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
@@ -284,7 +284,7 @@
val textDir = new File(dir, "text").getCanonicalPath
checkError(
exception = intercept[AnalysisException] {
- Seq(1).toDF.write.text(textDir)
+ Seq(1).toDF().write.text(textDir)
},
errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
@@ -295,7 +295,7 @@
checkError(
exception = intercept[AnalysisException] {
- Seq(1.2).toDF.write.text(textDir)
+ Seq(1.2).toDF().write.text(textDir)
},
errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
@@ -306,7 +306,7 @@
checkError(
exception = intercept[AnalysisException] {
- Seq(true).toDF.write.text(textDir)
+ Seq(true).toDF().write.text(textDir)
},
errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
@@ -350,7 +350,7 @@
)
// read path
- Seq("aaa").toDF.write.mode("overwrite").text(textDir)
+ Seq("aaa").toDF().write.mode("overwrite").text(textDir)
checkError(
exception = intercept[AnalysisException] {
val schema = StructType(StructField("a", IntegerType, true) :: Nil)
@@ -802,7 +802,7 @@
withTempPath { dir =>
val path = dir.getCanonicalPath
spark.range(10).write.orc(path)
- val row = spark.read.orc(path).select(input_file_name).first()
+ val row = spark.read.orc(path).select(input_file_name()).first()
assert(row.getString(0).contains(path))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala b/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala
index 6d5a0dc..8556ccf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala
@@ -223,7 +223,7 @@
// in case data has more than maxRecordsPerFile, split into multiple writers to improve
// datagen speed files will be truncated to maxRecordsPerFile value, so the final
// result will be the same.
- val numRows = data.count
+ val numRows = data.count()
val maxRecordPerFile = spark.conf.get(SQLConf.MAX_RECORDS_PER_FILE)
if (maxRecordPerFile > 0 && numRows > maxRecordPerFile) {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
index e228d47..c55d8b7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
@@ -442,7 +442,7 @@
test("SPARK-30998: Unsupported nested inner generators") {
checkError(
exception = intercept[AnalysisException] {
- sql("SELECT array(array(1, 2), array(3)) v").select(explode(explode($"v"))).collect
+ sql("SELECT array(array(1, 2), array(3)) v").select(explode(explode($"v"))).collect()
},
errorClass = "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS",
parameters = Map("expression" -> "\"explode(explode(v))\""))
@@ -509,12 +509,12 @@
}
test("SPARK-39061: inline should handle null struct") {
- testNullStruct
+ testNullStruct()
}
test("SPARK-39496: inline eval path should handle null struct") {
withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") {
- testNullStruct
+ testNullStruct()
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntervalFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntervalFunctionsSuite.scala
index c7e307b..0cbc15e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/IntervalFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/IntervalFunctionsSuite.scala
@@ -28,7 +28,7 @@
test("SPARK-36022: Respect interval fields in extract") {
yearMonthIntervalTypes.foreach { dtype =>
- val ymDF = Seq(Period.of(1, 2, 0)).toDF.select($"value" cast dtype as "value")
+ val ymDF = Seq(Period.of(1, 2, 0)).toDF().select($"value" cast dtype as "value")
.select($"value" cast dtype as "value")
val expectedMap = Map("year" -> 1, "month" -> 2)
YM.yearMonthFields.foreach { field =>
@@ -45,7 +45,7 @@
}
dayTimeIntervalTypes.foreach { dtype =>
- val dtDF = Seq(Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)).toDF
+ val dtDF = Seq(Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)).toDF()
.select($"value" cast dtype as "value")
val expectedMap = Map("day" -> 1, "hour" -> 2, "minute" -> 3, "second" -> 4)
DT.dayTimeFields.foreach { field =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index 51e66f4..5effa2e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -940,7 +940,7 @@
val json_tuple_result = Seq(s"""{"test":"$str"}""").toDF("json")
.withColumn("result", json_tuple($"json", "test"))
.select($"result")
- .as[String].head.length
+ .as[String].head().length
assert(json_tuple_result === len)
}
}
@@ -1284,7 +1284,7 @@
}
test("SPARK-35982: from_json/to_json for map types where value types are year-month intervals") {
- val ymDF = Seq(Period.of(1, 2, 0)).toDF
+ val ymDF = Seq(Period.of(1, 2, 0)).toDF()
Seq(
(YearMonthIntervalType(), """{"key":"INTERVAL '1-2' YEAR TO MONTH"}""", Period.of(1, 2, 0)),
(YearMonthIntervalType(YEAR), """{"key":"INTERVAL '1' YEAR"}""", Period.of(1, 0, 0)),
@@ -1308,7 +1308,7 @@
}
test("SPARK-35983: from_json/to_json for map types where value types are day-time intervals") {
- val dtDF = Seq(Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)).toDF
+ val dtDF = Seq(Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)).toDF()
Seq(
(DayTimeIntervalType(), """{"key":"INTERVAL '1 02:03:04' DAY TO SECOND"}""",
Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4)),
@@ -1350,7 +1350,7 @@
test("SPARK-36491: Make from_json/to_json to handle timestamp_ntz type properly") {
val localDT = LocalDateTime.parse("2021-08-12T15:16:23")
- val df = Seq(localDT).toDF
+ val df = Seq(localDT).toDF()
val toJsonDF = df.select(to_json(map(lit("key"), $"value")) as "json")
checkAnswer(toJsonDF, Row("""{"key":"2021-08-12T15:16:23.000"}"""))
val fromJsonDF = toJsonDF
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala
index cc4aeb4..a82c818 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala
@@ -899,14 +899,14 @@
test("Leaf expression as aggregate expressions should be eligible to lift up") {
// literal
sql(s"select 1, avg(salary) as m, m + 1 from $testTable group by dept")
- .queryExecution.assertAnalyzed
+ .queryExecution.assertAnalyzed()
// leaf expression current_date, now and etc
sql(s"select current_date(), max(salary) as m, m + 1 from $testTable group by dept")
- .queryExecution.assertAnalyzed
+ .queryExecution.assertAnalyzed()
sql("select dateadd(month, 5, current_date()), min(salary) as m, m + 1 as n " +
- s"from $testTable group by dept").queryExecution.assertAnalyzed
+ s"from $testTable group by dept").queryExecution.assertAnalyzed()
sql(s"select now() as n, dateadd(day, -1, n) from $testTable group by name")
- .queryExecution.assertAnalyzed
+ .queryExecution.assertAnalyzed()
}
test("Aggregate expressions containing no aggregate or grouping expressions still resolves") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ProductAggSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ProductAggSuite.scala
index 7fff530..0dbc40b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ProductAggSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ProductAggSuite.scala
@@ -69,7 +69,7 @@
val prodFactorials = data16.withColumn("f", product(col("x")).over(win)).orderBy(col("x"))
- assert(prodFactorials.count === 16)
+ assert(prodFactorials.count() === 16)
checkAnswer(
prodFactorials.limit(5),
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 30a6ec6..09db772 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -283,7 +283,7 @@
}
test("SPARK-43522: Fix creating struct column name with index of array") {
- val df = Seq("a=b,c=d,d=f").toDF.withColumn("key_value", split('value, ","))
+ val df = Seq("a=b,c=d,d=f").toDF().withColumn("key_value", split('value, ","))
.withColumn("map_entry", transform(col("key_value"), x => struct(split(x, "=")
.getItem(0), split(x, "=").getItem(1)))).select("map_entry")
@@ -2545,7 +2545,7 @@
test("SPARK-18053: ARRAY equality is broken") {
withTable("array_tbl") {
spark.range(10).select(array($"id").as("arr")).write.saveAsTable("array_tbl")
- assert(sql("SELECT * FROM array_tbl where arr = ARRAY(1L)").count == 1)
+ assert(sql("SELECT * FROM array_tbl where arr = ARRAY(1L)").count() == 1)
}
}
@@ -2881,7 +2881,7 @@
withTable("fact_stats", "dim_stats") {
val factData = Seq((1, 1, 99, 1), (2, 2, 99, 2), (3, 1, 99, 3), (4, 2, 99, 4))
val storeData = Seq((1, "BW", "DE"), (2, "AZ", "US"))
- spark.udf.register("filterND", udf((value: Int) => value > 2).asNondeterministic)
+ spark.udf.register("filterND", udf((value: Int) => value > 2).asNondeterministic())
factData.toDF("date_id", "store_id", "product_id", "units_sold")
.write.mode("overwrite").partitionBy("store_id").format("parquet").saveAsTable("fact_stats")
storeData.toDF("store_id", "state_province", "country")
@@ -2938,14 +2938,14 @@
val distributeExprs = (0 until 100).map(c => s"id$c").mkString(",")
df.selectExpr(columns : _*).createTempView("spark_25084")
assert(
- spark.sql(s"select * from spark_25084 distribute by ($distributeExprs)").count === count)
+ spark.sql(s"select * from spark_25084 distribute by ($distributeExprs)").count() === count)
}
}
test("SPARK-25144 'distinct' causes memory leak") {
- val ds = List(Foo(Some("bar"))).toDS
- val result = ds.flatMap(_.bar).distinct
- result.rdd.isEmpty
+ val ds = List(Foo(Some("bar"))).toDS()
+ val result = ds.flatMap(_.bar).distinct()
+ result.rdd.isEmpty()
}
test("SPARK-25454: decimal division with negative scale") {
@@ -4249,7 +4249,7 @@
test("SPARK-35749: Parse multiple unit fields interval literals as day-time interval types") {
def evalAsSecond(query: String): Long = {
- spark.sql(query).map(_.getAs[Duration](0)).collect.head.getSeconds
+ spark.sql(query).map(_.getAs[Duration](0)).collect().head.getSeconds
}
Seq(
@@ -4272,7 +4272,7 @@
test("SPARK-35749: Parse multiple unit fields interval literals as year-month interval types") {
def evalAsYearAndMonth(query: String): (Int, Int) = {
- val result = spark.sql(query).map(_.getAs[Period](0)).collect.head
+ val result = spark.sql(query).map(_.getAs[Period](0)).collect().head
(result.getYears, result.getMonths)
}
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 8df91c6..6a08acb 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
@@ -26,7 +26,7 @@
class SSBQuerySuite extends BenchmarkQueryTest {
override def beforeAll(): Unit = {
- super.beforeAll
+ super.beforeAll()
sql(
"""
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
index 3ca21a3..e621980 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
@@ -24,12 +24,12 @@
class SerializationSuite extends SparkFunSuite with SharedSparkSession {
test("[SPARK-5235] SQLContext should be serializable") {
- val spark = SparkSession.builder.getOrCreate()
+ val spark = SparkSession.builder().getOrCreate()
new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sqlContext)
}
test("[SPARK-26409] SQLConf should be serializable") {
- val spark = SparkSession.builder.getOrCreate()
+ val spark = SparkSession.builder().getOrCreate()
new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sessionState.conf)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala
index 0e7c294..90082c9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala
@@ -301,7 +301,7 @@
val error = intercept[SparkException] {
session.range(1).foreach { v =>
- SparkSession.builder.master("local").getOrCreate()
+ SparkSession.builder().master("local").getOrCreate()
()
}
}.getMessage()
@@ -313,7 +313,7 @@
val session = SparkSession.builder().master("local-cluster[3, 1, 1024]").getOrCreate()
session.range(1).foreach { v =>
- SparkSession.builder.master("local")
+ SparkSession.builder().master("local")
.config(EXECUTOR_ALLOW_SPARK_CONTEXT.key, true).getOrCreate().stop()
()
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
index 422498a..179f407 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
@@ -812,7 +812,7 @@
)
checkError(
exception = intercept[SparkRuntimeException] {
- sql("select regexp_extract('', '[a\\\\d]{0, 2}', 1)").collect
+ sql("select regexp_extract('', '[a\\\\d]{0, 2}', 1)").collect()
},
errorClass = "INVALID_PARAMETER_VALUE.PATTERN",
parameters = Map(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
index 4fab10c..d82fba8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
@@ -1197,7 +1197,7 @@
test("SPARK-23316: AnalysisException after max iteration reached for IN query") {
// before the fix this would throw AnalysisException
- spark.range(10).where("(id,id) in (select id, null from range(3))").count
+ spark.range(10).where("(id,id) in (select id, null from range(3))").count()
}
test("SPARK-24085 scalar subquery in partitioning expression") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
index 0670ade..814cf2f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
@@ -377,7 +377,7 @@
spark.udf.register("f", (a: Int) => a)
val outputStream = new java.io.ByteArrayOutputStream()
Console.withOut(outputStream) {
- spark.sql("SELECT f(a._1) FROM x").show
+ spark.sql("SELECT f(a._1) FROM x").show()
}
assert(outputStream.toString.contains("f(a._1)"))
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala
index 3fb5a4a..0fb5ad6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala
@@ -27,7 +27,7 @@
val df = Seq(
(1, 2, 3),
(4, 5, 6)
- ).toDF
+ ).toDF()
assert(SQLUtils.dfToCols(df) === Array(
Array(1, 4),
Array(2, 5),
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index 64a9411..3cbbc37 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -116,14 +116,14 @@
val df1 = sql(s"DESCRIBE $t id")
assert(df1.schema.map(field => (field.name, field.dataType))
=== Seq(("info_name", StringType), ("info_value", StringType)))
- assert(df1.collect === Seq(
+ assert(df1.collect() === Seq(
Row("col_name", "id"),
Row("data_type", "bigint"),
Row("comment", "NULL")))
val df2 = sql(s"DESCRIBE $t data")
assert(df2.schema.map(field => (field.name, field.dataType))
=== Seq(("info_name", StringType), ("info_value", StringType)))
- assert(df2.collect === Seq(
+ assert(df2.collect() === Seq(
Row("col_name", "data"),
Row("data_type", "string"),
Row("comment", "hello")))
@@ -149,7 +149,7 @@
val df = sql(s"DESCRIBE $t $col")
assert(df.schema.map(field => (field.name, field.dataType))
=== Seq(("info_name", StringType), ("info_value", StringType)))
- assert(df.collect === Seq(
+ assert(df.collect() === Seq(
Row("col_name", "id"),
Row("data_type", "bigint"),
Row("comment", "NULL")))
@@ -399,11 +399,11 @@
val describe = spark.sql(s"DESCRIBE $identifier")
val part1 = describe
.filter("col_name = 'Part 0'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(part1 === "id")
val part2 = describe
.filter("col_name = 'Part 1'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(part2 === "bucket(4, data1, data2, data3, data4)")
}
}
@@ -423,11 +423,11 @@
val describe = spark.sql(s"DESCRIBE $identifier")
val part1 = describe
.filter("col_name = 'Part 0'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(part1 === "id")
val part2 = describe
.filter("col_name = 'Part 1'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(part2 === "sorted_bucket(data1, data2, 4, data3, data4)")
}
}
@@ -441,7 +441,7 @@
"AS SELECT id FROM source")
val location = spark.sql(s"DESCRIBE EXTENDED $identifier")
.filter("col_name = 'Location'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(location === "file:/tmp/foo")
}
}
@@ -458,7 +458,7 @@
"AS SELECT id FROM source")
val location = spark.sql(s"DESCRIBE EXTENDED $identifier")
.filter("col_name = 'Location'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(location === "file:/tmp/foo")
}
}
@@ -1743,7 +1743,7 @@
.add("catalog", StringType, nullable = false)
.add("namespace", StringType, nullable = false)
val df = sql("SHOW CURRENT NAMESPACE")
- val rows = df.collect
+ val rows = df.collect()
assert(df.schema === schema)
assert(rows.length == 1)
@@ -1973,15 +1973,15 @@
val describe = spark.sql(s"DESCRIBE $identifier")
val part1 = describe
.filter("col_name = 'Part 0'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(part1 === "a")
val part2 = describe
.filter("col_name = 'Part 1'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(part2 === "b")
val part3 = describe
.filter("col_name = 'Part 2'")
- .select("data_type").head.getString(0)
+ .select("data_type").head().getString(0)
assert(part3 === "sorted_bucket(c, d, 4, e, f)")
}
}
@@ -2082,7 +2082,7 @@
parameters = Map(
"table" -> "testcat.ns1.ns2.tbl",
"filters" -> "[id = 2, id = id]"))
- assert(spark.table(t).count === 3)
+ assert(spark.table(t).count() === 3)
}
}
@@ -2775,18 +2775,18 @@
val df = sql("SHOW CATALOGS")
assert(df.schema === schema)
- assert(df.collect === Array(Row("spark_catalog")))
+ assert(df.collect() === Array(Row("spark_catalog")))
sql("use testcat")
sql("use testpart")
sql("use testcat2")
- assert(sql("SHOW CATALOGS").collect === Array(
+ assert(sql("SHOW CATALOGS").collect() === Array(
Row("spark_catalog"), Row("testcat"), Row("testcat2"), Row("testpart")))
- assert(sql("SHOW CATALOGS LIKE 'test*'").collect === Array(
+ assert(sql("SHOW CATALOGS LIKE 'test*'").collect() === Array(
Row("testcat"), Row("testcat2"), Row("testpart")))
- assert(sql("SHOW CATALOGS LIKE 'testcat*'").collect === Array(
+ assert(sql("SHOW CATALOGS LIKE 'testcat*'").collect() === Array(
Row("testcat"), Row("testcat2")))
}
@@ -2902,9 +2902,9 @@
sql(s"INSERT INTO $t2 VALUES (3)")
sql(s"INSERT INTO $t2 VALUES (4)")
- assert(sql("SELECT * FROM t VERSION AS OF 'Snapshot123456789'").collect
+ assert(sql("SELECT * FROM t VERSION AS OF 'Snapshot123456789'").collect()
=== Array(Row(1), Row(2)))
- assert(sql("SELECT * FROM t VERSION AS OF 2345678910").collect
+ assert(sql("SELECT * FROM t VERSION AS OF 2345678910").collect()
=== Array(Row(3), Row(4)))
}
@@ -2928,27 +2928,28 @@
sql(s"INSERT INTO $t4 VALUES (7)")
sql(s"INSERT INTO $t4 VALUES (8)")
- assert(sql("SELECT * FROM t TIMESTAMP AS OF '2019-01-29 00:37:58'").collect
+ assert(sql("SELECT * FROM t TIMESTAMP AS OF '2019-01-29 00:37:58'").collect()
=== Array(Row(5), Row(6)))
- assert(sql("SELECT * FROM t TIMESTAMP AS OF '2021-01-29 00:00:00'").collect
+ assert(sql("SELECT * FROM t TIMESTAMP AS OF '2021-01-29 00:00:00'").collect()
=== Array(Row(7), Row(8)))
- assert(sql(s"SELECT * FROM t TIMESTAMP AS OF $ts1InSeconds").collect
+ assert(sql(s"SELECT * FROM t TIMESTAMP AS OF $ts1InSeconds").collect()
=== Array(Row(5), Row(6)))
- assert(sql(s"SELECT * FROM t TIMESTAMP AS OF $ts2InSeconds").collect
+ assert(sql(s"SELECT * FROM t TIMESTAMP AS OF $ts2InSeconds").collect()
=== Array(Row(7), Row(8)))
- assert(sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts1InSeconds").collect
+ assert(sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts1InSeconds").collect()
=== Array(Row(5), Row(6)))
- assert(sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts2InSeconds").collect
+ assert(sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts2InSeconds").collect()
=== Array(Row(7), Row(8)))
- assert(sql("SELECT * FROM t TIMESTAMP AS OF make_date(2021, 1, 29)").collect
+ assert(sql("SELECT * FROM t TIMESTAMP AS OF make_date(2021, 1, 29)").collect()
=== Array(Row(7), Row(8)))
- assert(sql("SELECT * FROM t TIMESTAMP AS OF to_timestamp('2021-01-29 00:00:00')").collect
+ assert(sql("SELECT * FROM t TIMESTAMP AS OF to_timestamp('2021-01-29 00:00:00')").collect()
=== Array(Row(7), Row(8)))
// Scalar subquery is also supported.
- assert(sql("SELECT * FROM t TIMESTAMP AS OF (SELECT make_date(2021, 1, 29))").collect
+ assert(sql("SELECT * FROM t TIMESTAMP AS OF (SELECT make_date(2021, 1, 29))").collect()
=== Array(Row(7), Row(8)))
// Nested subquery also works
- assert(sql("SELECT * FROM t TIMESTAMP AS OF (SELECT (SELECT make_date(2021, 1, 29)))").collect
+ assert(
+ sql("SELECT * FROM t TIMESTAMP AS OF (SELECT (SELECT make_date(2021, 1, 29)))").collect()
=== Array(Row(7), Row(8)))
checkError(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala
index 82a7d11..eeef056 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala
@@ -80,7 +80,7 @@
sql(s"DELETE FROM $t WHERE id IN (SELECT id FROM $t)")
}
- assert(spark.table(t).count === 3)
+ assert(spark.table(t).count() === 3)
assert(exc.getMessage.contains("Delete by condition with subquery is not supported"))
}
}
@@ -94,7 +94,7 @@
sql(s"DELETE FROM $t WHERE id > 3 AND p > 3")
}
- assert(spark.table(t).count === 3)
+ assert(spark.table(t).count() === 3)
assert(exc.getMessage.contains(s"Cannot delete from table $t"))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
index d133270..78bbabb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
@@ -103,7 +103,7 @@
val encryptedEmptyText24 = "9RDK70sHNzqAFRcpfGM5gQ=="
val encryptedEmptyText32 = "j9IDsCvlYXtcVJUf4FAjQQ=="
- val df1 = Seq("Spark", "").toDF
+ val df1 = Seq("Spark", "").toDF()
val df2 = Seq(
(encryptedText16, encryptedText24, encryptedText32),
(encryptedEmptyText16, encryptedEmptyText24, encryptedEmptyText32)
@@ -117,7 +117,7 @@
def checkInvalidKeyLength(df: => DataFrame, inputBytes: Int): Unit = {
checkError(
exception = intercept[SparkException] {
- df.collect
+ df.collect()
}.getCause.asInstanceOf[SparkRuntimeException],
errorClass = "INVALID_PARAMETER_VALUE.AES_KEY_LENGTH",
parameters = Map(
@@ -154,7 +154,7 @@
("value32", "12345678123456781234567812345678")).foreach { case (colName, key) =>
checkError(
exception = intercept[SparkException] {
- df2.selectExpr(s"aes_decrypt(unbase64($colName), binary('$key'), 'ECB')").collect
+ df2.selectExpr(s"aes_decrypt(unbase64($colName), binary('$key'), 'ECB')").collect()
}.getCause.asInstanceOf[SparkRuntimeException],
errorClass = "INVALID_PARAMETER_VALUE.AES_CRYPTO_ERROR",
parameters = Map("parameter" -> "`expr`, `key`",
@@ -172,7 +172,7 @@
def checkUnsupportedMode(df: => DataFrame, mode: String, padding: String): Unit = {
checkError(
exception = intercept[SparkException] {
- df.collect
+ df.collect()
}.getCause.asInstanceOf[SparkRuntimeException],
errorClass = "UNSUPPORTED_FEATURE.AES_MODE",
parameters = Map("mode" -> mode,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
index bfbbf2f..a07d206 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
@@ -614,7 +614,7 @@
).toDF("a", "b")
df.createTempView("v")
- if (defaultSerDe == "hive-serde") {
+ if (defaultSerDe() == "hive-serde") {
checkAnswer(sql(
"""
|SELECT TRANSFORM(a, b)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala
index 5265095..bacd098 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala
@@ -44,7 +44,7 @@
def checkDumpedPlans(path: String, expected: Int): Unit = Utils.tryWithResource(
Source.fromFile(path)) { source =>
- assert(source.getLines.toList
+ assert(source.getLines().toList
.takeWhile(_ != "== Whole Stage Codegen ==") == List(
"== Parsed Logical Plan ==",
s"Range (0, $expected, step=1, splits=Some(2))",
@@ -252,20 +252,20 @@
val showTables = ShowTables(UnresolvedNamespace(Seq.empty[String]), None)
val showTablesQe = qe(showTables, mockCallback1)
- showTablesQe.assertAnalyzed
- mockCallback1.assertAnalyzed
+ showTablesQe.assertAnalyzed()
+ mockCallback1.assertAnalyzed()
assert(showTablesQe.commandExecuted.isInstanceOf[CommandResult])
- mockCallback1.assertCommandExecuted
+ mockCallback1.assertCommandExecuted()
assert(showTablesQe.executedPlan.isInstanceOf[CommandResultExec])
val showTablesResultExec = showTablesQe.executedPlan.asInstanceOf[CommandResultExec]
assert(showTablesResultExec.commandPhysicalPlan.isInstanceOf[ShowTablesExec])
val project = Project(showTables.output, SubqueryAlias("s", showTables))
val projectQe = qe(project, mockCallback2)
- projectQe.assertAnalyzed
- mockCallback2.assertAnalyzed
+ projectQe.assertAnalyzed()
+ mockCallback2.assertAnalyzed()
assert(projectQe.commandExecuted.isInstanceOf[Project])
- mockCallback2.assertCommandExecuted
+ mockCallback2.assertCommandExecuted()
assert(projectQe.commandExecuted.children.length == 1)
assert(projectQe.commandExecuted.children(0).isInstanceOf[SubqueryAlias])
assert(projectQe.commandExecuted.children(0).children.length == 1)
@@ -284,28 +284,28 @@
showTables,
new QueryPlanningTracker(Some(mockCallback)),
CommandExecutionMode.SKIP)
- showTablesQe.assertAnalyzed
- mockCallback.assertAnalyzed
- showTablesQe.assertOptimized
- mockCallback.assertOptimized
- showTablesQe.assertSparkPlanPrepared
- mockCallback.assertSparkPlanPrepared
- showTablesQe.assertExecutedPlanPrepared
- mockCallback.assertExecutedPlanPrepared
+ showTablesQe.assertAnalyzed()
+ mockCallback.assertAnalyzed()
+ showTablesQe.assertOptimized()
+ mockCallback.assertOptimized()
+ showTablesQe.assertSparkPlanPrepared()
+ mockCallback.assertSparkPlanPrepared()
+ showTablesQe.assertExecutedPlanPrepared()
+ mockCallback.assertExecutedPlanPrepared()
}
test("SPARK-44145: Plan setReadyForExecution") {
val mockCallback = MockCallback()
val plan: LogicalPlan = org.apache.spark.sql.catalyst.plans.logical.Range(0, 1, 1, 1)
val df = Dataset.ofRows(spark, plan, new QueryPlanningTracker(Some(mockCallback)))
- df.queryExecution.assertAnalyzed
- mockCallback.assertAnalyzed
- df.queryExecution.assertOptimized
- mockCallback.assertOptimized
- df.queryExecution.assertSparkPlanPrepared
- mockCallback.assertSparkPlanPrepared
- df.queryExecution.assertExecutedPlanPrepared
- mockCallback.assertExecutedPlanPrepared
+ df.queryExecution.assertAnalyzed()
+ mockCallback.assertAnalyzed()
+ df.queryExecution.assertOptimized()
+ mockCallback.assertOptimized()
+ df.queryExecution.assertSparkPlanPrepared()
+ mockCallback.assertSparkPlanPrepared()
+ df.queryExecution.assertExecutedPlanPrepared()
+ mockCallback.assertExecutedPlanPrepared()
}
test("SPARK-35378: Return UnsafeRow in CommandResultExecCheck execute methods") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
index 9e9d717..b5bac80 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
@@ -246,7 +246,7 @@
val df =
spark.read.format(format).load(path.getCanonicalPath).filter($"i" > 0).orderBy($"i")
- assert(df.collect === Array(Row(1, 1), Row(2, 2)))
+ assert(df.collect() === Array(Row(1, 1), Row(2, 2)))
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala
index 766f495..48860f3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala
@@ -49,7 +49,7 @@
}
test("concurrent query execution with fork-join pool (SPARK-13747)") {
- val spark = SparkSession.builder
+ val spark = SparkSession.builder()
.master("local[*]")
.appName("test")
.getOrCreate()
@@ -69,7 +69,7 @@
* Trigger SPARK-10548 by mocking a parent and its child thread executing queries concurrently.
*/
private def testConcurrentQueryExecution(sc: SparkContext): Unit = {
- val spark = SparkSession.builder.getOrCreate()
+ val spark = SparkSession.builder().getOrCreate()
import spark.implicits._
// Initialize local properties. This is necessary for the test to pass.
@@ -103,7 +103,7 @@
test("Finding QueryExecution for given executionId") {
- val spark = SparkSession.builder.master("local[*]").appName("test").getOrCreate()
+ val spark = SparkSession.builder().master("local[*]").appName("test").getOrCreate()
import spark.implicits._
var queryExecution: QueryExecution = null
@@ -205,7 +205,7 @@
}
test("SPARK-44591: jobTags property") {
- val spark = SparkSession.builder.master("local[*]").appName("test").getOrCreate()
+ val spark = SparkSession.builder().master("local[*]").appName("test").getOrCreate()
val jobTag = "jobTag"
try {
spark.sparkContext.addJobTag(jobTag)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
index 5a413c7..08db88b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
@@ -743,7 +743,7 @@
p.isInstanceOf[WholeStageCodegenExec] &&
p.asInstanceOf[WholeStageCodegenExec].codegenStageId == 0),
"codegen stage IDs should be preserved through ReuseExchange")
- checkAnswer(join, df.toDF)
+ checkAnswer(join, df.toDF())
}
}
@@ -825,7 +825,7 @@
// Case1: LocalTableScanExec is the root of a query plan tree.
// In this case, WholeStageCodegenExec should not be inserted
// as the direct parent of LocalTableScanExec.
- val df = Seq(1, 2, 3).toDF
+ val df = Seq(1, 2, 3).toDF()
val rootOfExecutedPlan = df.queryExecution.executedPlan
// Ensure WholeStageCodegenExec is not inserted and
@@ -836,7 +836,7 @@
// Case2: The parent of a LocalTableScanExec supports WholeStageCodegen.
// In this case, the LocalTableScanExec should be within a WholeStageCodegen domain
// and no more InputAdapter is inserted as the direct parent of the LocalTableScanExec.
- val aggregatedDF = Seq(1, 2, 3).toDF.groupBy("value").sum()
+ val aggregatedDF = Seq(1, 2, 3).toDF().groupBy("value").sum()
val executedPlan = aggregatedDF.queryExecution.executedPlan
// HashAggregateExec supports WholeStageCodegen and it's the parent of
@@ -863,7 +863,7 @@
// Tet case with keys
"SELECT k, AVG(v) FROM VALUES((1, 1)) t(k, v) GROUP BY k").foreach { query =>
val e = intercept[IllegalStateException] {
- sql(query).collect
+ sql(query).collect()
}
assert(e.getMessage.contains(expectedErrMsg))
}
@@ -885,7 +885,7 @@
"SELECT k, AVG(a + b), SUM(a + b + c) FROM VALUES((1, 1, 1, 1)) t(k, a, b, c) " +
"GROUP BY k").foreach { query =>
val e = intercept[IllegalStateException] {
- sql(query).collect
+ sql(query).collect()
}
assert(e.getMessage.contains(expectedErrMsg))
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
index 406b3ed..0ced0f8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
@@ -1311,8 +1311,8 @@
SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10",
SQLConf.SHUFFLE_PARTITIONS.key -> "10") {
- val df1 = spark.range(10).toDF.repartitionByRange($"id".asc)
- val df2 = spark.range(10).toDF.repartitionByRange(($"id" + 1).asc)
+ val df1 = spark.range(10).toDF().repartitionByRange($"id".asc)
+ val df2 = spark.range(10).toDF().repartitionByRange(($"id" + 1).asc)
val partitionsNum1 = df1.rdd.collectPartitions().length
val partitionsNum2 = df2.rdd.collectPartitions().length
@@ -1344,7 +1344,7 @@
SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10",
SQLConf.SHUFFLE_PARTITIONS.key -> "10") {
- spark.range(10).toDF.createTempView("test")
+ spark.range(10).toDF().createTempView("test")
val df1 = spark.sql("SELECT /*+ REPARTITION(id) */ * from test")
val df2 = spark.sql("SELECT /*+ REPARTITION_BY_RANGE(id) */ * from test")
@@ -2851,7 +2851,7 @@
val aggDf1 = emptyDf.agg(sum("id").as("id")).withColumn("name", lit("df1"))
val aggDf2 = emptyDf.agg(sum("id").as("id")).withColumn("name", lit("df2"))
val unionDF = aggDf1.union(aggDf2)
- checkAnswer(unionDF.select("id").distinct, Seq(Row(null)))
+ checkAnswer(unionDF.select("id").distinct(), Seq(Row(null)))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala
index 4a0c88b..eea2cb0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala
@@ -78,7 +78,7 @@
groupingSchema,
updateInputRow,
mergeAggBuffer,
- createNewAggregationBuffer)
+ createNewAggregationBuffer())
(5000 to 100000).foreach { _ =>
randomKV(inputRow, group)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala
index 1cb9874..523da0d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala
@@ -40,7 +40,7 @@
private val scaleFactor = 100
private val N = scaleFactor * 1000 * 1000
- private val df = spark.range(N).map(_ => Random.nextInt)
+ private val df = spark.range(N).map(_ => Random.nextInt())
private def writeORCBenchmark(): Unit = {
withTempPath { dir =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala
index 62bd85d..771f944 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala
@@ -55,7 +55,7 @@
.setIfMissing("spark.driver.memory", "3g")
.setIfMissing("spark.executor.memory", "3g")
- val sparkSession = SparkSession.builder.config(conf).getOrCreate()
+ val sparkSession = SparkSession.builder().config(conf).getOrCreate()
// Set default configs. Individual cases will change them if necessary.
sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
@@ -135,7 +135,7 @@
withTempPath { dir =>
withTempTable("t1", "csvTable", "jsonTable", "parquetV1Table", "parquetV2Table", "orcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
@@ -280,7 +280,7 @@
withTempPath { dir =>
withTempTable("t1", "parquetV1Table", "parquetV2Table", "orcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(dir,
spark.sql(s"SELECT named_struct('f', CAST(value as ${dataType.sql})) as col FROM t1"),
@@ -338,7 +338,7 @@
withTempPath { dir =>
withTempTable("t1", "parquetV1Table", "parquetV2Table", "orcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).map { x =>
+ spark.range(values).map(_ => Random.nextLong()).map { x =>
val arrayOfStructColumn = (0 until 5).map(i => (x + i, s"$x" * 5))
val mapOfStructColumn = Map(
s"$x" -> (x * 0.1, (x, s"$x" * 100)),
@@ -404,7 +404,7 @@
withTempPath { dir =>
withTempTable("t1", "csvTable", "jsonTable", "parquetV1Table", "parquetV2Table", "orcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(
dir,
@@ -453,7 +453,7 @@
withTempPath { dir =>
withTempTable("t1", "csvTable", "jsonTable", "parquetV1Table", "parquetV2Table", "orcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(
dir,
@@ -502,7 +502,7 @@
withTempPath { dir =>
withTempTable("t1", "csvTable", "jsonTable", "parquetV1Table", "parquetV2Table", "orcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
@@ -703,7 +703,7 @@
import spark.implicits._
val middle = width / 2
val selectExpr = (1 to width).map(i => s"value as c$i")
- spark.range(values).map(_ => Random.nextLong).toDF()
+ spark.range(values).map(_ => Random.nextLong()).toDF()
.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT * FROM t1"))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala
index b572444..4862571 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala
@@ -74,7 +74,7 @@
} else {
monotonically_increasing_id()
}
- val df = spark.range(numRows).map(_ => Random.nextLong).selectExpr(selectExpr: _*)
+ val df = spark.range(numRows).map(_ => Random.nextLong()).selectExpr(selectExpr: _*)
.withColumn("value", valueCol)
.sort("value")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala
index ad7850e..796c56d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala
@@ -126,7 +126,7 @@
import spark.implicits._
val df = spark.sparkContext.parallelize(Seq(("1",
Array.fill(M)({
- val i = math.random
+ val i = math.random()
(i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString)
})))).toDF("col", "arr")
@@ -139,7 +139,7 @@
import spark.implicits._
val df = spark.sparkContext.parallelize(Seq(("1",
Array.fill(M)({
- val i = math.random
+ val i = math.random()
(i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString)
})))).toDF("col", "arr")
.selectExpr("col", "struct(col, arr) as st")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala
index c967615..a09a64d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala
@@ -33,7 +33,7 @@
object PrimitiveArrayBenchmark extends SqlBasedBenchmark {
override def getSparkSession: SparkSession = {
- SparkSession.builder
+ SparkSession.builder()
.master("local[1]")
.appName("microbenchmark")
.config("spark.sql.shuffle.partitions", 1)
@@ -54,24 +54,24 @@
val sc = spark.sparkContext
val primitiveIntArray = Array.fill[Int](count)(65535)
- val dsInt = sc.parallelize(Seq(primitiveIntArray), 1).toDS
- dsInt.count // force to build dataset
+ val dsInt = sc.parallelize(Seq(primitiveIntArray), 1).toDS()
+ dsInt.count() // force to build dataset
val intArray = { i: Int =>
var n = 0
var len = 0
while (n < iters) {
- len += dsInt.map(e => e).queryExecution.toRdd.collect.length
+ len += dsInt.map(e => e).queryExecution.toRdd.collect().length
n += 1
}
}
val primitiveDoubleArray = Array.fill[Double](count)(65535.0)
- val dsDouble = sc.parallelize(Seq(primitiveDoubleArray), 1).toDS
- dsDouble.count // force to build dataset
+ val dsDouble = sc.parallelize(Seq(primitiveDoubleArray), 1).toDS()
+ dsDouble.count() // force to build dataset
val doubleArray = { i: Int =>
var n = 0
var len = 0
while (n < iters) {
- len += dsDouble.map(e => e).queryExecution.toRdd.collect.length
+ len += dsDouble.map(e => e).queryExecution.toRdd.collect().length
n += 1
}
}
@@ -79,6 +79,6 @@
val benchmark = new Benchmark("Write an array in Dataset", count * iters, output = output)
benchmark.addCase("Int ")(intArray)
benchmark.addCase("Double")(doubleArray)
- benchmark.run
+ benchmark.run()
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala
index fe89916..c26272d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala
@@ -60,7 +60,7 @@
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.set("spark.kryo.registrationRequired", "true")
- SparkSession.builder.config(conf).getOrCreate()
+ SparkSession.builder().config(conf).getOrCreate()
}
val tables = Seq("catalog_page", "catalog_returns", "customer", "customer_address",
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala
index 9b09564..d84fa5e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala
@@ -49,12 +49,12 @@
val count = 1024 * 1024 * 16
val rand = new Random(42)
val intArrayToRow = intEncoder.createSerializer()
- val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt }
+ val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt() }
val intUnsafeArray = intArrayToRow(intPrimitiveArray).getArray(0)
val readIntArray = { i: Int =>
var n = 0
while (n < iters) {
- val len = intUnsafeArray.numElements
+ val len = intUnsafeArray.numElements()
var sum = 0
var i = 0
while (i < len) {
@@ -65,13 +65,13 @@
}
}
- val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble }
+ val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble() }
val doubleArrayToRow = doubleEncoder.createSerializer()
val doubleUnsafeArray = doubleArrayToRow(doublePrimitiveArray).getArray(0)
val readDoubleArray = { i: Int =>
var n = 0
while (n < iters) {
- val len = doubleUnsafeArray.numElements
+ val len = doubleUnsafeArray.numElements()
var sum = 0.0
var i = 0
while (i < len) {
@@ -85,7 +85,7 @@
val benchmark = new Benchmark("Read UnsafeArrayData", count * iters, output = output)
benchmark.addCase("Int")(readIntArray)
benchmark.addCase("Double")(readDoubleArray)
- benchmark.run
+ benchmark.run()
}
def writeUnsafeArray(iters: Int): Unit = {
@@ -93,7 +93,7 @@
val rand = new Random(42)
var intTotalLength: Int = 0
- val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt }
+ val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt() }
val intArrayToRow = intEncoder.createSerializer()
val writeIntArray = { i: Int =>
var len = 0
@@ -106,7 +106,7 @@
}
var doubleTotalLength: Int = 0
- val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble }
+ val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble() }
val doubleArrayToRow = doubleEncoder.createSerializer()
val writeDoubleArray = { i: Int =>
var len = 0
@@ -121,7 +121,7 @@
val benchmark = new Benchmark("Write UnsafeArrayData", count * iters, output = output)
benchmark.addCase("Int")(writeIntArray)
benchmark.addCase("Double")(writeDoubleArray)
- benchmark.run
+ benchmark.run()
}
def getPrimitiveArray(iters: Int): Unit = {
@@ -129,28 +129,28 @@
val rand = new Random(42)
var intTotalLength: Int = 0
- val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt }
+ val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt() }
val intArrayToRow = intEncoder.createSerializer()
val intUnsafeArray = intArrayToRow(intPrimitiveArray).getArray(0)
val readIntArray = { i: Int =>
var len = 0
var n = 0
while (n < iters) {
- len += intUnsafeArray.toIntArray.length
+ len += intUnsafeArray.toIntArray().length
n += 1
}
intTotalLength = len
}
var doubleTotalLength: Int = 0
- val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble }
+ val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble() }
val doubleArrayToRow = doubleEncoder.createSerializer()
val doubleUnsafeArray = doubleArrayToRow(doublePrimitiveArray).getArray(0)
val readDoubleArray = { i: Int =>
var len = 0
var n = 0
while (n < iters) {
- len += doubleUnsafeArray.toDoubleArray.length
+ len += doubleUnsafeArray.toDoubleArray().length
n += 1
}
doubleTotalLength = len
@@ -160,7 +160,7 @@
new Benchmark("Get primitive array from UnsafeArrayData", count * iters, output = output)
benchmark.addCase("Int")(readIntArray)
benchmark.addCase("Double")(readDoubleArray)
- benchmark.run
+ benchmark.run()
}
def putPrimitiveArray(iters: Int): Unit = {
@@ -168,7 +168,7 @@
val rand = new Random(42)
var intTotalLen: Int = 0
- val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt }
+ val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt() }
val createIntArray = { i: Int =>
var len = 0
var n = 0
@@ -180,7 +180,7 @@
}
var doubleTotalLen: Int = 0
- val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble }
+ val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble() }
val createDoubleArray = { i: Int =>
var len = 0
var n = 0
@@ -195,7 +195,7 @@
new Benchmark("Create UnsafeArrayData from primitive array", count * iters, output = output)
benchmark.addCase("Int")(createIntArray)
benchmark.addCase("Double")(createDoubleArray)
- benchmark.run
+ benchmark.run()
}
override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
index eb8d41e..2ef4985 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
@@ -89,7 +89,7 @@
for (width <- widthsToTest) {
val selectExpr = (1 to width).map(i => s"id as a_$i")
benchmark.addCase(s"$width select expressions") { iter =>
- spark.range(1).toDF.selectExpr(selectExpr: _*)
+ spark.range(1).toDF().selectExpr(selectExpr: _*)
}
}
benchmark.run()
@@ -114,7 +114,7 @@
// normalize by width to keep constant data size
val numRows = scaleFactor / width
val selectExpr = (1 to width).map(i => s"id as a_$i")
- val df = spark.range(numRows).toDF.selectExpr(selectExpr: _*).cache()
+ val df = spark.range(numRows).toDF().selectExpr(selectExpr: _*).cache()
df.count() // force caching
addCases(benchmark, df, s"$width cols x $numRows rows", "a_1")
}
@@ -209,7 +209,7 @@
for (width <- widthsToTest) {
val numRows = scaleFactor / width
val datum = Tuple1((1 to width).map(i => ("value_" + i -> 1)).toMap)
- val df = spark.range(numRows).map(_ => datum).toDF.cache()
+ val df = spark.range(numRows).map(_ => datum).toDF().cache()
df.count() // force caching
addCases(benchmark, df, s"$width wide x $numRows rows", "_1[\"value_1\"]")
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
index de04938..78f8372 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
@@ -167,9 +167,9 @@
test("access only some column of the all of columns") {
val df = spark.range(1, 100).map(i => (i, (i + 1).toFloat)).toDF("i", "f")
- df.cache
- df.count // forced to build cache
- assert(df.filter("f <= 10.0").count == 9)
+ df.cache()
+ df.count() // forced to build cache
+ assert(df.filter("f <= 10.0").count() == 9)
}
test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") {
@@ -365,7 +365,7 @@
}
test("cached row count should be calculated") {
- val data = spark.range(6).toDF
+ val data = spark.range(6).toDF()
val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5),
MEMORY_ONLY, plan, None, data.logicalPlan)
@@ -390,7 +390,7 @@
}
val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data)))
val df = spark.createDataFrame(rdd, StructType(schemas))
- val row = df.persist.take(1).apply(0)
+ val row = df.persist().take(1).apply(0)
checkAnswer(df, row)
}
}
@@ -410,7 +410,7 @@
)
val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data)))
val df = spark.createDataFrame(rdd, StructType(schemas))
- val row = df.persist.take(1).apply(0)
+ val row = df.persist().take(1).apply(0)
checkAnswer(df, row)
}
}
@@ -432,16 +432,16 @@
)
val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data)))
val df = spark.createDataFrame(rdd, StructType(schemas))
- val row = df.persist.take(1).apply(0)
+ val row = df.persist().take(1).apply(0)
checkAnswer(df, row)
}
}
test("InMemoryTableScanExec should return correct output ordering and partitioning") {
- val df1 = Seq((0, 0), (1, 1)).toDF
- .repartition(col("_1")).sortWithinPartitions(col("_1")).persist
- val df2 = Seq((0, 0), (1, 1)).toDF
- .repartition(col("_1")).sortWithinPartitions(col("_1")).persist
+ val df1 = Seq((0, 0), (1, 1)).toDF()
+ .repartition(col("_1")).sortWithinPartitions(col("_1")).persist()
+ val df2 = Seq((0, 0), (1, 1)).toDF()
+ .repartition(col("_1")).sortWithinPartitions(col("_1")).persist()
// Because two cached dataframes have the same logical plan, this is a self-join actually.
// So we force one of in-memory relation to alias its output. Then we can test if original and
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala
index 7c5df7f..bfc32a7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala
@@ -126,7 +126,7 @@
.toDF("k", "v")
.where("k='Properties'")
.where("v=''")
- .count == 1, s"$key is a reserved namespace property and ignored")
+ .count() == 1, s"$key is a reserved namespace property and ignored")
val meta =
getCatalog(catalog).asNamespaceCatalog.loadNamespaceMetadata(namespaceArray)
assert(meta.get(key) == null || !meta.get(key).contains("foo"),
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
index b019381..418bff6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
@@ -167,11 +167,11 @@
val catalog1 = new InMemoryFileIndex(
spark, Seq(unqualifiedDirPath), Map.empty, None)
- assert(catalog1.allFiles.map(_.getPath) === Seq(qualifiedFilePath))
+ assert(catalog1.allFiles().map(_.getPath) === Seq(qualifiedFilePath))
val catalog2 = new InMemoryFileIndex(
spark, Seq(unqualifiedFilePath), Map.empty, None)
- assert(catalog2.allFiles.map(_.getPath) === Seq(qualifiedFilePath))
+ assert(catalog2.allFiles().map(_.getPath) === Seq(qualifiedFilePath))
}
}
@@ -541,7 +541,7 @@
when(dfs.listLocatedStatus(path)).thenReturn(new RemoteIterator[LocatedFileStatus] {
val iter = statuses.iterator
override def hasNext: Boolean = iter.hasNext
- override def next(): LocatedFileStatus = iter.next
+ override def next(): LocatedFileStatus = iter.next()
})
val fileIndex = new TestInMemoryFileIndex(spark, path)
assert(fileIndex.leafFileStatuses.toSeq == statuses)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceAggregatePushDownSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceAggregatePushDownSuite.scala
index 317abd5..67ec3cc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceAggregatePushDownSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceAggregatePushDownSuite.scala
@@ -212,7 +212,7 @@
" min(id), p FROM tmp group by p"
var expected = Array.empty[Row]
withSQLConf(aggPushDownEnabledKey -> "false") {
- expected = sql(query).collect
+ expected = sql(query).collect()
}
Seq("false", "true").foreach { enableVectorizedReader =>
withSQLConf(aggPushDownEnabledKey -> "true",
@@ -246,7 +246,7 @@
" p4, p2, p3, p1 FROM tmp GROUP BY p1, p2, p3, p4"
var expected = Array.empty[Row]
withSQLConf(aggPushDownEnabledKey -> "false") {
- expected = sql(query).collect
+ expected = sql(query).collect()
}
Seq("false", "true").foreach { enableVectorizedReader =>
withSQLConf(aggPushDownEnabledKey -> "true",
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PrunePartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PrunePartitionSuiteBase.scala
index 430e9f8..0263ae5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PrunePartitionSuiteBase.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PrunePartitionSuiteBase.scala
@@ -94,9 +94,10 @@
val plan = qe.sparkPlan
assert(getScanExecPartitionSize(plan) == expectedPartitionCount)
- val collectFn: PartialFunction[SparkPlan, Seq[Expression]] = collectPartitionFiltersFn orElse {
- case BatchScanExec(_, scan: FileScan, _, _, _, _) => scan.partitionFilters
- }
+ val collectFn: PartialFunction[SparkPlan, Seq[Expression]] =
+ collectPartitionFiltersFn() orElse {
+ case BatchScanExec(_, scan: FileScan, _, _, _, _) => scan.partitionFilters
+ }
val pushedDownPartitionFilters = plan.collectFirst(collectFn)
.map(exps => exps.filterNot(e => e.isInstanceOf[IsNotNull]))
val pushedFilters = pushedDownPartitionFilters.map(filters => {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
index bf496d6..0a0b23d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
@@ -966,7 +966,7 @@
| {"itemId": 2, "itemData": "b"}
|]}
|""".stripMargin
- val df = spark.read.json(Seq(jsonStr).toDS)
+ val df = spark.read.json(Seq(jsonStr).toDS())
makeDataSourceFile(df, new File(path))
spark.read.format(dataSourceName).load(path)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index 38fbf46..111e88d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -95,7 +95,7 @@
val numRows = if (withHeader) numCars else numCars + 1
// schema
assert(df.schema.fieldNames.length === numColumns)
- assert(df.count === numRows)
+ assert(df.count() === numRows)
if (checkHeader) {
if (withHeader) {
@@ -405,7 +405,7 @@
.schema(StructType(List(StructField("column", StringType, false))))
.load(testFile(emptyFile))
- assert(result.collect.size === 0)
+ assert(result.collect().size === 0)
assert(result.schema.fieldNames.size === 1)
}
@@ -1441,7 +1441,7 @@
.option("multiLine", multiLine)
.schema(schema.add(columnNameOfCorruptRecord, IntegerType))
.csv(testFile(valueMalformedFile))
- .collect
+ .collect()
}.getMessage
assert(errMsg.startsWith("The field for corrupt records must be string type and nullable"))
}
@@ -1721,7 +1721,7 @@
.option("inferSchema", true).option("samplingRatio", 0.1)
.option("path", path.getCanonicalPath)
.format("csv")
- .load
+ .load()
assert(readback2.schema == new StructType().add("_c0", IntegerType))
}
})
@@ -2328,12 +2328,12 @@
test("lineSep restrictions") {
val errMsg1 = intercept[IllegalArgumentException] {
- spark.read.option("lineSep", "").csv(testFile(carsFile)).collect
+ spark.read.option("lineSep", "").csv(testFile(carsFile)).collect()
}.getMessage
assert(errMsg1.contains("'lineSep' cannot be an empty string"))
val errMsg2 = intercept[IllegalArgumentException] {
- spark.read.option("lineSep", "123").csv(testFile(carsFile)).collect
+ spark.read.option("lineSep", "123").csv(testFile(carsFile)).collect()
}.getMessage
assert(errMsg2.contains("'lineSep' can contain only 1 character"))
}
@@ -2374,7 +2374,7 @@
test("SPARK-26208: write and read empty data to csv file with headers") {
withTempPath { path =>
- val df1 = spark.range(10).repartition(2).filter(_ < 0).map(_.toString).toDF
+ val df1 = spark.range(10).repartition(2).filter(_ < 0).map(_.toString).toDF()
// we have 2 partitions but they are both empty and will be filtered out upon writing
// thanks to SPARK-23271 one new empty partition will be inserted
df1.write.format("csv").option("header", true).save(path.getAbsolutePath)
@@ -2407,7 +2407,7 @@
assert(spark.read
.option("delimiter", "|")
.option("inferSchema", "true")
- .csv(Seq("1,2").toDS).schema.head.dataType === StringType)
+ .csv(Seq("1,2").toDS()).schema.head.dataType === StringType)
}
test("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") {
@@ -2651,7 +2651,7 @@
test("SPARK-32025: infer the schema from mixed-type values") {
withTempPath { path =>
- Seq("col_mixed_types", "2012", "1997", "True").toDS.write.text(path.getCanonicalPath)
+ Seq("col_mixed_types", "2012", "1997", "True").toDS().write.text(path.getCanonicalPath)
val df = spark.read.format("csv")
.option("header", "true")
.option("inferSchema", "true")
@@ -2663,7 +2663,7 @@
test("SPARK-32614: don't treat rows starting with null char as comment") {
withTempPath { path =>
- Seq("\u0000foo", "bar", "baz").toDS.write.text(path.getCanonicalPath)
+ Seq("\u0000foo", "bar", "baz").toDS().write.text(path.getCanonicalPath)
val df = spark.read.format("csv")
.option("header", "false")
.option("inferSchema", "true")
@@ -2713,7 +2713,7 @@
spark.range(3).coalesce(1).write.csv(s"$basePath/$csvTableName")
val readback = spark.read
.csv(s"$basePath/${"""(\[|\]|\{|\})""".r.replaceAllIn(csvTableName, """\\$1""")}")
- assert(readback.collect sameElements Array(Row("0"), Row("1"), Row("2")))
+ assert(readback.collect() sameElements Array(Row("0"), Row("1"), Row("2")))
}
}
@@ -2745,7 +2745,7 @@
val bufSize = 128
val line = "X" * (bufSize - 1) + "| |"
withTempPath { path =>
- Seq(line).toDF.write.text(path.getAbsolutePath)
+ Seq(line).toDF().write.text(path.getAbsolutePath)
assert(spark.read.format("csv")
.option("delimiter", "|")
.option("ignoreTrailingWhiteSpace", "true").load(path.getAbsolutePath).count() == 1)
@@ -2777,7 +2777,7 @@
StructType(
StructField("f1", StringType, nullable = false) ::
StructField("f2", StringType, nullable = false) :: Nil)
- ).option("mode", "DROPMALFORMED").csv(Seq("a,", "a,b").toDS),
+ ).option("mode", "DROPMALFORMED").csv(Seq("a,", "a,b").toDS()),
Row("a", "b"))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
index 5096b24..f0561a3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
@@ -1252,7 +1252,7 @@
val df1 = spark.createDataFrame(rowRDD1, schema1)
df1.createOrReplaceTempView("applySchema1")
- val df2 = df1.toDF
+ val df2 = df1.toDF()
val result = df2.toJSON.collect()
// scalastyle:off
assert(result(0) === "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}")
@@ -1275,7 +1275,7 @@
val df3 = spark.createDataFrame(rowRDD2, schema2)
df3.createOrReplaceTempView("applySchema2")
- val df4 = df3.toDF
+ val df4 = df3.toDF()
val result2 = df4.toJSON.collect()
assert(result2(1) === "{\"f1\":{\"f11\":2,\"f12\":false},\"f2\":{\"B2\":null}}")
@@ -1584,7 +1584,7 @@
// inferring partitions because the original path in the "path" option will list the
// partition directory that has been removed.
assert(
- spark.read.options(extraOptions).format("json").option("path", path).load.count() === 2)
+ spark.read.options(extraOptions).format("json").option("path", path).load().count() === 2)
}
}
}
@@ -1663,7 +1663,7 @@
.format("json")
.load(jsonDir)
- assert(jsonCopy.count == jsonDF.count)
+ assert(jsonCopy.count() == jsonDF.count())
val jsonCopySome = jsonCopy.selectExpr("string", "long", "boolean")
val jsonDFSome = jsonDF.selectExpr("string", "long", "boolean")
checkAnswer(jsonCopySome, jsonDFSome)
@@ -1701,7 +1701,7 @@
.options(extraOptions)
.load(jsonDir)
- assert(jsonCopy.count == jsonDF.count)
+ assert(jsonCopy.count() == jsonDF.count())
val jsonCopySome = jsonCopy.selectExpr("string", "long", "boolean")
val jsonDFSome = jsonDF.selectExpr("string", "long", "boolean")
checkAnswer(jsonCopySome, jsonDFSome)
@@ -2075,7 +2075,7 @@
.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
.schema(schema)
.json(path)
- .collect
+ .collect()
}.getMessage
assert(errMsg.startsWith("The field for corrupt records must be string type and nullable"))
}
@@ -2235,7 +2235,7 @@
// inferred when sampling ratio is involved.
val readback2 = spark.read
.option("samplingRatio", 0.1).option("path", path.getCanonicalPath)
- .format("json").load
+ .format("json").load()
assert(readback2.schema == new StructType().add("f1", LongType))
}
})
@@ -2662,7 +2662,7 @@
private def failedOnEmptyString(dataType: DataType): Unit = {
val df = spark.read.schema(s"a ${dataType.catalogString}")
- .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS)
+ .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS())
val e = intercept[SparkException] {df.collect()}
checkError(
exception = e.getCause.getCause.getCause.asInstanceOf[SparkRuntimeException],
@@ -2673,7 +2673,7 @@
private def emptyString(dataType: DataType, expected: Any): Unit = {
val df = spark.read.schema(s"a ${dataType.catalogString}")
- .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS)
+ .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS())
checkAnswer(df, Row(expected) :: Nil)
}
@@ -2699,7 +2699,7 @@
test("SPARK-25040: allowing empty strings when legacy config is enabled") {
def emptyStringAsNull(dataType: DataType): Unit = {
val df = spark.read.schema(s"a ${dataType.catalogString}")
- .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS)
+ .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS())
checkAnswer(df, Row(null) :: Nil)
}
@@ -2741,7 +2741,7 @@
test("inferring timestamp type") {
def schemaOf(jsons: String*): StructType = {
- spark.read.option("inferTimestamp", true).json(jsons.toDS).schema
+ spark.read.option("inferTimestamp", true).json(jsons.toDS()).schema
}
assert(schemaOf(
@@ -3127,7 +3127,7 @@
spark.range(3).coalesce(1).write.json(s"$basePath/$jsonTableName")
val readback = spark.read
.json(s"$basePath/${"""(\[|\]|\{|\})""".r.replaceAllIn(jsonTableName, """\\$1""")}")
- assert(readback.collect sameElements Array(Row(0), Row(1), Row(2)))
+ assert(readback.collect() sameElements Array(Row(0), Row(1), Row(2)))
}
}
@@ -3136,7 +3136,7 @@
withTempPaths(2) { paths =>
paths.foreach(_.delete())
val seq = Seq("a", "\n", "\u3042")
- val df = seq.toDF
+ val df = seq.toDF()
val basePath1 = paths(0).getCanonicalPath
df.write.option("writeNonAsciiCharacterAsCodePoint", "true")
@@ -3190,7 +3190,7 @@
withSQLConf(SQLConf.LEAF_NODE_DEFAULT_PARALLELISM.key -> "1") {
withTempPath { path =>
val basePath = path.getCanonicalPath
- val df = Seq("a", "b", "c").toDF
+ val df = Seq("a", "b", "c").toDF()
df.write.option("pretty", "true").json(basePath)
val expectedText =
@@ -3241,7 +3241,7 @@
StructType(
StructField("f1", LongType, nullable = false) ::
StructField("f2", LongType, nullable = false) :: Nil)
- ).option("mode", "DROPMALFORMED").json(Seq("""{"f1": 1}""").toDS),
+ ).option("mode", "DROPMALFORMED").json(Seq("""{"f1": 1}""").toDS()),
// It is for testing legacy configuration. This is technically a bug as
// `0` has to be `null` but the schema is non-nullable.
Row(1, 0))
@@ -3251,7 +3251,7 @@
test("SPARK-36379: proceed parsing with root nulls in permissive mode") {
val exception = intercept[SparkException] {
spark.read.option("mode", "failfast")
- .schema("a string").json(Seq("""[{"a": "str"}, null]""").toDS).collect()
+ .schema("a string").json(Seq("""[{"a": "str"}, null]""").toDS()).collect()
}
assert(exception.getMessage.contains("Malformed records are detected"))
@@ -3265,7 +3265,7 @@
// Here, since an array fails to parse in the middle, we will return one row.
checkAnswer(
spark.read.option("mode", "permissive")
- .json(Seq("""[{"a": "str"}, null, {"a": "str"}]""").toDS),
+ .json(Seq("""[{"a": "str"}, null, {"a": "str"}]""").toDS()),
Row(null) :: Nil)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
index f12f882..39447ed 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
@@ -413,7 +413,7 @@
// results. So, this checks if the number of result is less than the original count
// of data, and then checks if it contains the expected data.
assert(
- sourceDf.count < 10 && expectedData.subsetOf(data),
+ sourceDf.count() < 10 && expectedData.subsetOf(data),
s"No data was filtered for predicate: $pred")
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
index 024f5f6..2ed2494 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
@@ -366,7 +366,7 @@
test("SPARK-24322 Fix incorrect workaround for bug in java.sql.Timestamp") {
withTempPath { path =>
val ts = Timestamp.valueOf("1900-05-05 12:34:56.000789")
- Seq(ts).toDF.write.orc(path.getCanonicalPath)
+ Seq(ts).toDF().write.orc(path.getCanonicalPath)
checkAnswer(spark.read.orc(path.getCanonicalPath), Row(ts))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala
index 65b067e..a0d11e2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala
@@ -58,7 +58,7 @@
test("All Types Dictionary") {
(1 :: 1000 :: Nil).foreach { n => {
withTempPath { dir =>
- List.fill(n)(ROW).toDF.repartition(1).write.parquet(dir.getCanonicalPath)
+ List.fill(n)(ROW).toDF().repartition(1).write.parquet(dir.getCanonicalPath)
val file = TestUtils.listDirectory(dir).head
val conf = sqlContext.conf
@@ -87,7 +87,7 @@
test("All Types Null") {
(1 :: 100 :: Nil).foreach { n => {
withTempPath { dir =>
- val data = List.fill(n)(NULL_ROW).toDF
+ val data = List.fill(n)(NULL_ROW).toDF()
data.repartition(1).write.parquet(dir.getCanonicalPath)
val file = TestUtils.listDirectory(dir).head
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala
index c10e179..2465dee 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala
@@ -148,7 +148,7 @@
assert(df.select("*", s"${FileFormat.METADATA_NAME}.$mixedCaseRowIndex")
.where(s"$EXPECTED_ROW_ID_COL != $mixedCaseRowIndex")
- .count == 0)
+ .count() == 0)
}
}
}
@@ -160,7 +160,7 @@
Seq(StructField(ROW_INDEX_TEMPORARY_COLUMN_NAME, LongType))) { df =>
assert(df
.where(col(EXPECTED_ROW_ID_COL) === col(ROW_INDEX_TEMPORARY_COLUMN_NAME))
- .count == NUM_ROWS)
+ .count() == NUM_ROWS)
}
// File format not supporting row index generation populates missing column with nulls.
@@ -168,7 +168,7 @@
Seq(StructField(ROW_INDEX_TEMPORARY_COLUMN_NAME, LongType))) { df =>
assert(df
.where(col(ROW_INDEX_TEMPORARY_COLUMN_NAME).isNull)
- .count == NUM_ROWS)
+ .count() == NUM_ROWS)
}
}
@@ -180,7 +180,7 @@
// ROW_INDEX_TEMPORARY_COLUMN_NAME in their schemas.
assert(df
.where(col(EXPECTED_ROW_ID_COL) === col(ROW_INDEX_TEMPORARY_COLUMN_NAME))
- .count == NUM_ROWS)
+ .count() == NUM_ROWS)
// Column cannot be read in combination with _metadata.row_index.
intercept[AnalysisException](df.select("*", FileFormat.METADATA_NAME).collect())
@@ -194,7 +194,7 @@
// Column values are set for each partition, rather than populated with generated row indexes.
assert(df
.where(col(EXPECTED_PARTITION_COL) === col(ROW_INDEX_TEMPORARY_COLUMN_NAME))
- .count == NUM_ROWS)
+ .count() == NUM_ROWS)
// Column cannot be read in combination with _metadata.row_index.
intercept[AnalysisException](df.select("*", FileFormat.METADATA_NAME).collect())
@@ -230,13 +230,13 @@
assert(spark
.read.parquet(dir.getAbsolutePath)
- .count == NUM_ROWS)
+ .count() == NUM_ROWS)
// The _metadata.row_index is returning data from the file, not generated metadata.
assert(spark
.read.parquet(dir.getAbsolutePath)
.select(s"${FileFormat.METADATA_NAME}.${ROW_INDEX}")
- .distinct.count == NUM_ROWS / 10)
+ .distinct().count() == NUM_ROWS / 10)
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
index 269a3ef..0d64d8f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
@@ -791,7 +791,7 @@
SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> INT96.toString) {
import testImplicits._
withTempPath { file =>
- millisData.map(i => Tuple1(Timestamp.valueOf(i))).toDF
+ millisData.map(i => Tuple1(Timestamp.valueOf(i))).toDF()
.write.format(dataSourceName).save(file.getCanonicalPath)
readParquetFile(file.getCanonicalPath) { df =>
val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema)
@@ -1024,7 +1024,7 @@
// When a filter is pushed to Parquet, Parquet can apply it to every row.
// So, we can check the number of rows returned from the Parquet
// to make sure our filter pushdown work.
- assert(stripSparkFilter(df).count == 1)
+ assert(stripSparkFilter(df).count() == 1)
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
index 2e7b261..dc8a89c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
@@ -964,7 +964,7 @@
withAllParquetReaders {
withTempPath { path =>
// Repeated values for dictionary encoding.
- Seq(Some("A"), Some("A"), None).toDF.repartition(1)
+ Seq(Some("A"), Some("A"), None).toDF().repartition(1)
.write.parquet(path.getAbsolutePath)
val df = spark.read.parquet(path.getAbsolutePath)
checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), df)
@@ -1305,7 +1305,7 @@
override def userClass: Class[TestArray] = classOf[TestArray]
override def deserialize(datum: Any): TestArray = datum match {
- case value: ArrayData => TestArray(value.toLongArray.toSeq)
+ case value: ArrayData => TestArray(value.toLongArray().toSeq)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
index 69b07e6..4bfc34a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
@@ -759,7 +759,7 @@
val res = new UnsafeRow(1)
val it = map.get(1L, res)
assert(it.hasNext)
- assert(it.next.getLong(0) == 1)
+ assert(it.next().getLong(0) == 1)
assert(it.hasNext != ignoresDuplicatedKey)
map.free()
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala
index 81667d5..cae008a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala
@@ -36,7 +36,7 @@
protected def currentExecutionIds(): Set[Long] = {
spark.sparkContext.listenerBus.waitUntilEmpty(10000)
- statusStore.executionsList.map(_.executionId).toSet
+ statusStore.executionsList().map(_.executionId).toSet
}
protected def statusStore: SQLAppStatusStore = spark.sharedState.statusStore
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala
index 84c23b7..3101281 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala
@@ -98,7 +98,7 @@
df.count()
val statusStore = spark.sharedState.statusStore
- val lastExecId = statusStore.executionsList.last.executionId
+ val lastExecId = statusStore.executionsList().last.executionId
val executionMetrics = statusStore.execution(lastExecId).get.metrics.mkString
for (metric <- pythonSQLMetrics) {
assert(executionMetrics.contains(metric))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala
index d083cac..efb7e12 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala
@@ -147,7 +147,7 @@
def startQuery(): StreamingQuery = {
ds.writeStream
.foreachBatch((ds: Dataset[Row], batchId: Long) => {
- ds.collect.foreach((row: Row) => {
+ ds.collect().foreach((row: Row) => {
data += row.getInt(0)
}: Unit)
countDownLatch.countDown()
@@ -352,7 +352,7 @@
ds.writeStream
.trigger(trigger)
.foreachBatch((ds: Dataset[Row], batchId: Long) => {
- ds.collect.foreach((row: Row) => {
+ ds.collect().foreach((row: Row) => {
data += row.getInt(0)
}: Unit)
countDownLatch.countDown()
@@ -512,7 +512,7 @@
ds.writeStream
.trigger(trigger)
.foreachBatch((ds: Dataset[Row], batchId: Long) => {
- ds.collect.foreach((row: Row) => {
+ ds.collect().foreach((row: Row) => {
data += row.getInt(0)
}: Unit)
countDownLatch.countDown()
@@ -785,7 +785,7 @@
def startQuery(): StreamingQuery = {
ds.writeStream
.foreachBatch((ds: Dataset[Row], batchId: Long) => {
- ds.collect.foreach((row: Row) => {
+ ds.collect().foreach((row: Row) => {
data += row.getInt(0)
}: Unit)
})
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
index cbcb4a4..cdf736b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
@@ -122,7 +122,7 @@
SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key ->
classOf[CreateAtomicTestManager].getName) {
val fileManager =
- CheckpointFileManager.create(new Path("/"), spark.sessionState.newHadoopConf)
+ CheckpointFileManager.create(new Path("/"), spark.sessionState.newHadoopConf())
assert(fileManager.isInstanceOf[CreateAtomicTestManager])
}
}
@@ -236,5 +236,5 @@
}
private object CheckpointFileManagerSuiteFileSystem {
- val scheme = s"CheckpointFileManagerSuiteFileSystem${math.abs(Random.nextInt)}"
+ val scheme = s"CheckpointFileManagerSuiteFileSystem${math.abs(Random.nextInt())}"
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala
index d6707e7..d2b751d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala
@@ -331,7 +331,7 @@
}
object CountOpenLocalFileSystem {
- val scheme = s"FileStreamSinkLogSuite${math.abs(Random.nextInt)}fs"
+ val scheme = s"FileStreamSinkLogSuite${math.abs(Random.nextInt())}fs"
val pathToNumOpenCalled = new ConcurrentHashMap[String, JLong]
def resetCount(): Unit = pathToNumOpenCalled.clear()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala
index c0fd3fe3..110b562 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala
@@ -53,7 +53,7 @@
override def run(): Unit = {
executor.execute(() => {
// Record the trigger time, increment clock if needed and
- triggerTimes.add(clock.getTimeMillis.toInt)
+ triggerTimes.add(clock.getTimeMillis().toInt)
clock.advance(clockIncrementInTrigger)
clockIncrementInTrigger = 0 // reset this so that there are no runaway triggers
continueExecuting
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala
index b92fa4c..b27aa37 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala
@@ -34,7 +34,7 @@
test("foreachBatch with non-stateful query") {
val mem = MemoryStream[Int]
- val ds = mem.toDS.map(_ + 1)
+ val ds = mem.toDS().map(_ + 1)
val tester = new ForeachBatchTester[Int](mem)
val writer = (ds: Dataset[Int], batchId: Long) => tester.record(batchId, ds.map(_ + 1))
@@ -47,7 +47,7 @@
test("foreachBatch with non-stateful query - untyped Dataset") {
val mem = MemoryStream[Int]
- val ds = mem.toDF.selectExpr("value + 1 as value")
+ val ds = mem.toDF().selectExpr("value + 1 as value")
val tester = new ForeachBatchTester[Row](mem)(ExpressionEncoder(ds.schema))
val writer = (df: DataFrame, batchId: Long) =>
@@ -66,7 +66,7 @@
.select($"value" % 2 as "key")
.groupBy("key")
.agg(count("*") as "value")
- .toDF.as[KV]
+ .toDF().as[KV]
val tester = new ForeachBatchTester[KV](mem)
val writer = (batchDS: Dataset[KV], batchId: Long) => tester.record(batchId, batchDS)
@@ -84,7 +84,7 @@
.select($"value" % 2 as "key")
.groupBy("key")
.agg(count("*") as "value")
- .toDF.as[KV]
+ .toDF().as[KV]
val tester = new ForeachBatchTester[KV](mem)
val writer = (batchDS: Dataset[KV], batchId: Long) => tester.record(batchId, batchDS)
@@ -98,7 +98,7 @@
test("foreachBatch with batch specific operations") {
val mem = MemoryStream[Int]
- val ds = mem.toDS.map(_ + 1)
+ val ds = mem.toDS().map(_ + 1)
val tester = new ForeachBatchTester[Int](mem)
val writer: (Dataset[Int], Long) => Unit = { case (df, batchId) =>
@@ -127,7 +127,7 @@
test("foreachBatchSink does not affect metric generation") {
val mem = MemoryStream[Int]
- val ds = mem.toDS.map(_ + 1)
+ val ds = mem.toDS().map(_ + 1)
val tester = new ForeachBatchTester[Int](mem)
val writer = (ds: Dataset[Int], batchId: Long) => tester.record(batchId, ds.map(_ + 1))
@@ -139,7 +139,7 @@
}
test("throws errors in invalid situations") {
- val ds = MemoryStream[Int].toDS
+ val ds = MemoryStream[Int].toDS()
val ex1 = intercept[IllegalArgumentException] {
ds.writeStream.foreachBatch(null.asInstanceOf[(Dataset[Int], Long) => Unit]).start()
}
@@ -176,7 +176,7 @@
// typed
val mem = MemoryStream[Int]
- val ds = mem.toDS.map(_ + 1)
+ val ds = mem.toDS().map(_ + 1)
assertPlan(mem, ds)
// untyped
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala
index b75da90..48cc17b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala
@@ -83,8 +83,8 @@
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]
- val df1 = input1.toDF.select($"value" as "key", ($"value" * 2) as "leftValue")
- val df2 = input2.toDF.select($"value" as "key", ($"value" * 3) as "rightValue")
+ val df1 = input1.toDF().select($"value" as "key", ($"value" * 2) as "leftValue")
+ val df2 = input2.toDF().select($"value" as "key", ($"value" * 3) as "rightValue")
val joined = df1.join(df2, "key")
testStream(joined)(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala
index 7adafe1..28c755b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala
@@ -40,7 +40,7 @@
val conf = Map(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
classOf[RocksDBStateStoreProvider].getName)
- testStream(input.toDF.groupBy().count(), outputMode = OutputMode.Update)(
+ testStream(input.toDF().groupBy().count(), outputMode = OutputMode.Update)(
StartStream(checkpointLocation = dir.getAbsolutePath, additionalConfs = conf),
AddData(input, 1, 2, 3),
CheckAnswer(3),
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala
index 82f677a..4ce344d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala
@@ -75,7 +75,7 @@
test("RocksDB confs are passed correctly from SparkSession to db instance") {
val sparkConf = new SparkConf().setMaster("local").setAppName(this.getClass.getSimpleName)
- withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark =>
+ withSparkSession(SparkSession.builder().config(sparkConf).getOrCreate()) { spark =>
// Set the session confs that should be passed into RocksDB
val testConfs = Seq(
("spark.sql.streaming.stateStore.providerClass",
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala
index b5e1ecc..15d35ba 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala
@@ -1100,7 +1100,7 @@
db.load(0)
db.put("a", "1")
db.commit()
- db.getWriteBufferManagerAndCache
+ db.getWriteBufferManagerAndCache()
}
val remoteDir2 = dir2.getCanonicalPath
@@ -1108,7 +1108,7 @@
db.load(0)
db.put("a", "1")
db.commit()
- db.getWriteBufferManagerAndCache
+ db.getWriteBufferManagerAndCache()
}
if (boundedMemoryUsage == "true") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala
index c2728b9..f2a5554 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala
@@ -58,8 +58,8 @@
}
test("versioning and immutability") {
- withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark =>
- val path = Utils.createDirectory(tempDir, Random.nextFloat.toString).toString
+ withSparkSession(SparkSession.builder().config(sparkConf).getOrCreate()) { spark =>
+ val path = Utils.createDirectory(tempDir, Random.nextFloat().toString).toString
val rdd1 = makeRDD(spark.sparkContext, Seq(("a", 0), ("b", 0), ("a", 0)))
.mapPartitionsWithStateStore(spark.sqlContext, operatorStateInfo(path, version = 0),
keySchema, valueSchema, numColsPrefixKey = 0)(increment)
@@ -77,7 +77,7 @@
}
test("recovering from files") {
- val path = Utils.createDirectory(tempDir, Random.nextFloat.toString).toString
+ val path = Utils.createDirectory(tempDir, Random.nextFloat().toString).toString
def makeStoreRDD(
spark: SparkSession,
@@ -90,22 +90,22 @@
}
// Generate RDDs and state store data
- withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark =>
+ withSparkSession(SparkSession.builder().config(sparkConf).getOrCreate()) { spark =>
for (i <- 1 to 20) {
require(makeStoreRDD(spark, Seq(("a", 0)), i - 1).collect().toSet === Set(("a", 0) -> i))
}
}
// With a new context, try using the earlier state store data
- withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark =>
+ withSparkSession(SparkSession.builder().config(sparkConf).getOrCreate()) { spark =>
assert(makeStoreRDD(spark, Seq(("a", 0)), 20).collect().toSet === Set(("a", 0) -> 21))
}
}
test("usage with iterators - only gets and only puts") {
- withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark =>
+ withSparkSession(SparkSession.builder().config(sparkConf).getOrCreate()) { spark =>
implicit val sqlContext = spark.sqlContext
- val path = Utils.createDirectory(tempDir, Random.nextFloat.toString).toString
+ val path = Utils.createDirectory(tempDir, Random.nextFloat().toString).toString
val opId = 0
// Returns an iterator of the incremented value made into the store
@@ -158,9 +158,9 @@
quietly {
val queryRunId = UUID.randomUUID
val opId = 0
- val path = Utils.createDirectory(tempDir, Random.nextFloat.toString).toString
+ val path = Utils.createDirectory(tempDir, Random.nextFloat().toString).toString
- withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark =>
+ withSparkSession(SparkSession.builder().config(sparkConf).getOrCreate()) { spark =>
implicit val sqlContext = spark.sqlContext
val coordinatorRef = sqlContext.streams.stateStoreCoordinator
val storeProviderId1 = StateStoreProviderId(StateStoreId(path, opId, 0), queryRunId)
@@ -194,11 +194,11 @@
quietly {
withSparkSession(
- SparkSession.builder
+ SparkSession.builder()
.config(sparkConf.setMaster("local-cluster[2, 1, 1024]"))
.getOrCreate()) { spark =>
implicit val sqlContext = spark.sqlContext
- val path = Utils.createDirectory(tempDir, Random.nextFloat.toString).toString
+ val path = Utils.createDirectory(tempDir, Random.nextFloat().toString).toString
val opId = 0
val rdd1 = makeRDD(spark.sparkContext, Seq(("a", 0), ("b", 0), ("a", 0)))
.mapPartitionsWithStateStore(sqlContext, operatorStateInfo(path, version = 0),
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala
index 067a1a3..fd7a292 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala
@@ -133,7 +133,7 @@
}
test("failure after committing with MAX_BATCHES_TO_RETAIN_IN_MEMORY set to 1") {
- tryWithProviderResource(newStoreProvider(opId = Random.nextInt, partition = 0,
+ tryWithProviderResource(newStoreProvider(opId = Random.nextInt(), partition = 0,
numOfVersToRetainInMemory = 1)) { provider =>
var currentVersion = 0
@@ -174,7 +174,7 @@
}
test("no cache data with MAX_BATCHES_TO_RETAIN_IN_MEMORY set to 0") {
- tryWithProviderResource(newStoreProvider(opId = Random.nextInt, partition = 0,
+ tryWithProviderResource(newStoreProvider(opId = Random.nextInt(), partition = 0,
numOfVersToRetainInMemory = 0)) { provider =>
var currentVersion = 0
@@ -194,7 +194,7 @@
}
test("cleaning") {
- tryWithProviderResource(newStoreProvider(opId = Random.nextInt, partition = 0,
+ tryWithProviderResource(newStoreProvider(opId = Random.nextInt(), partition = 0,
minDeltasForSnapshot = 5)) { provider =>
for (i <- 1 to 20) {
@@ -221,7 +221,7 @@
conf.set("fs.defaultFS", "fake:///")
tryWithProviderResource(
- newStoreProvider(opId = Random.nextInt, partition = 0, hadoopConf = conf)) { provider =>
+ newStoreProvider(opId = Random.nextInt(), partition = 0, hadoopConf = conf)) { provider =>
provider.getStore(0).commit()
provider.getStore(0).commit()
@@ -234,7 +234,7 @@
}
test("corrupted file handling") {
- tryWithProviderResource(newStoreProvider(opId = Random.nextInt, partition = 0,
+ tryWithProviderResource(newStoreProvider(opId = Random.nextInt(), partition = 0,
minDeltasForSnapshot = 5)) { provider =>
for (i <- 1 to 6) {
@@ -280,7 +280,7 @@
errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_DELTA_FILE_NOT_EXISTS",
parameters = Map(
"fileToRead" -> s"${provider.stateStoreId.storeCheckpointLocation()}/1.delta",
- "clazz" -> s"${provider.toString}"
+ "clazz" -> s"${provider.toString()}"
)
)
}
@@ -660,7 +660,7 @@
classOf[CreateAtomicTestManager].getName)
val remoteDir = Utils.createTempDir().getAbsolutePath
- tryWithProviderResource(newStoreProvider(opId = Random.nextInt, partition = 0,
+ tryWithProviderResource(newStoreProvider(opId = Random.nextInt(), partition = 0,
dir = remoteDir, hadoopConf = hadoopConf)) { provider =>
// Disable failure of output stream and generate versions
@@ -1043,14 +1043,14 @@
put(store, "b", 0, 2)
// Updates should work while iterating of filtered entries
- val filtered = store.iterator.filter { tuple => keyRowToData(tuple.key) == ("a", 0) }
+ val filtered = store.iterator().filter { tuple => keyRowToData(tuple.key) == ("a", 0) }
filtered.foreach { tuple =>
store.put(tuple.key, dataToValueRow(valueRowToData(tuple.value) + 1))
}
assert(get(store, "a", 0) === Some(2))
// Removes should work while iterating of filtered entries
- val filtered2 = store.iterator.filter { tuple => keyRowToData(tuple.key) == ("b", 0) }
+ val filtered2 = store.iterator().filter { tuple => keyRowToData(tuple.key) == ("b", 0) }
filtered2.foreach { tuple => store.remove(tuple.key) }
assert(get(store, "b", 0) === None)
}
@@ -1252,7 +1252,7 @@
withCoordinatorRef(sc) { coordinatorRef =>
val dir = newDir()
val storeId = StateStoreProviderId(StateStoreId(dir, 0, 0), UUID.randomUUID)
- val storeConf = getDefaultStoreConf
+ val storeConf = getDefaultStoreConf()
val hadoopConf = new Configuration()
// Verify that trying to get incorrect versions throw errors
@@ -1616,5 +1616,5 @@
}
object RenameReturnsFalseFileSystem {
- val scheme = s"StateStoreSuite${math.abs(Random.nextInt)}fs"
+ val scheme = s"StateStoreSuite${math.abs(Random.nextInt())}fs"
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala
index 252bcea8..b194ce8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala
@@ -128,8 +128,8 @@
info.setAccumulables(accumulables)
- val start = SparkListenerTaskStart(stageInfo.stageId, stageInfo.attemptNumber, info)
- val end = SparkListenerTaskEnd(stageInfo.stageId, stageInfo.attemptNumber,
+ val start = SparkListenerTaskStart(stageInfo.stageId, stageInfo.attemptNumber(), info)
+ val end = SparkListenerTaskEnd(stageInfo.stageId, stageInfo.attemptNumber(),
taskType = "",
reason = null,
info,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
index 67206e9..9a3313b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
@@ -694,7 +694,7 @@
SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan),
time,
Map.empty))
- assert(statusStore.executionsCount === 2)
+ assert(statusStore.executionsCount() === 2)
assert(statusStore.execution(2) === None)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala
index 42125c5..3cbf0bb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala
@@ -512,7 +512,7 @@
}
withVectors(16, dataType) { testVector =>
- val columnAccessor = ColumnAccessor(dataType, columnBuilder.build)
+ val columnAccessor = ColumnAccessor(dataType, columnBuilder.build())
ColumnAccessor.decompress(columnAccessor, testVector, 16)
assert(testVector.isNullAt(0))
@@ -536,7 +536,7 @@
}
withVectors(16, dataType) { testVector =>
- val columnAccessor = ColumnAccessor(dataType, columnBuilder.build)
+ val columnAccessor = ColumnAccessor(dataType, columnBuilder.build())
ColumnAccessor.decompress(columnAccessor, testVector, 16)
assert(testVector.isNullAt(0))
@@ -560,7 +560,7 @@
}
withVectors(16, dataType) { testVector =>
- val columnAccessor = ColumnAccessor(dataType, columnBuilder.build)
+ val columnAccessor = ColumnAccessor(dataType, columnBuilder.build())
ColumnAccessor.decompress(columnAccessor, testVector, 16)
assert(testVector.isNullAt(0))
@@ -584,7 +584,7 @@
}
withVectors(16, dataType) { testVector =>
- val columnAccessor = ColumnAccessor(dataType, columnBuilder.build)
+ val columnAccessor = ColumnAccessor(dataType, columnBuilder.build())
ColumnAccessor.decompress(columnAccessor, testVector, 16)
assert(testVector.isNullAt(0))
@@ -608,7 +608,7 @@
}
withVectors(16, dataType) { testVector =>
- val columnAccessor = ColumnAccessor(dataType, columnBuilder.build)
+ val columnAccessor = ColumnAccessor(dataType, columnBuilder.build())
ColumnAccessor.decompress(columnAccessor, testVector, 16)
assert(testVector.isNullAt(0))
@@ -633,7 +633,7 @@
}
withVectors(16, dataType) { testVector =>
- val columnAccessor = ColumnAccessor(dataType, columnBuilder.build)
+ val columnAccessor = ColumnAccessor(dataType, columnBuilder.build())
ColumnAccessor.decompress(columnAccessor, testVector, 16)
assert(testVector.isNullAt(0))
@@ -657,7 +657,7 @@
}
withVectors(16, dataType) { testVector =>
- val columnAccessor = ColumnAccessor(dataType, columnBuilder.build)
+ val columnAccessor = ColumnAccessor(dataType, columnBuilder.build())
ColumnAccessor.decompress(columnAccessor, testVector, 16)
assert(testVector.isNullAt(0))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala
index bffd006..a11b209 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala
@@ -371,7 +371,7 @@
val benchmark = new Benchmark("String Read/Write", count * iters, output = output)
benchmark.addCase("On Heap")(column(MemoryMode.ON_HEAP))
benchmark.addCase("Off Heap")(column(MemoryMode.OFF_HEAP))
- benchmark.run
+ benchmark.run()
}
def arrayAccess(iters: Int): Unit = {
@@ -442,7 +442,7 @@
benchmark.addCase("On Heap Read Elements") { _ => readArrayElements(true) }
benchmark.addCase("Off Heap Read Elements") { _ => readArrayElements(false) }
- benchmark.run
+ benchmark.run()
}
override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala
index 81bcf7d..6149f3e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala
@@ -141,7 +141,7 @@
Seq(true)
.toDF()
.mapPartitions { _ =>
- if (TaskContext.get.getLocalProperty(confKey) == confValue) {
+ if (TaskContext.get().getLocalProperty(confKey) == confValue) {
Iterator(true)
} else {
Iterator.empty
@@ -173,7 +173,7 @@
def generateBroadcastDataFrame(confKey: String, confValue: String): Dataset[Boolean] = {
val df = spark.range(1).mapPartitions { _ =>
- Iterator(TaskContext.get.getLocalProperty(confKey) == confValue)
+ Iterator(TaskContext.get().getLocalProperty(confKey) == confValue)
}
df.hint("broadcast")
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 60785e3..5f28164 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -365,12 +365,12 @@
// This is a test to reflect discussion in SPARK-12218.
// The older versions of spark have this kind of bugs in parquet data source.
val df1 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2) OR NOT (NAME != 'mary')")
- assert(df1.collect.toSet === Set(Row("mary", 2)))
+ assert(df1.collect().toSet === Set(Row("mary", 2)))
// SPARK-22548: Incorrect nested AND expression pushed down to JDBC data source
val df2 = sql("SELECT * FROM foobar " +
"WHERE (THEID > 0 AND TRIM(NAME) = 'mary') OR (NAME = 'fred')")
- assert(df2.collect.toSet === Set(Row("fred", 1), Row("mary", 2)))
+ assert(df2.collect().toSet === Set(Row("fred", 1), Row("mary", 2)))
assert(checkNotPushdown(sql("SELECT * FROM foobar WHERE (THEID + 1) < 2")).collect().size == 0)
assert(checkNotPushdown(sql("SELECT * FROM foobar WHERE (THEID + 2) != 4")).collect().size == 2)
@@ -383,7 +383,7 @@
// are applied for columns with Filter producing wrong results. On the other hand, JDBCRDD
// correctly handles this case by assigning `requiredColumns` properly. See PR 10427 for more
// discussions.
- assert(sql("SELECT COUNT(1) FROM foobar WHERE NAME = 'mary'").collect.toSet === Set(Row(1)))
+ assert(sql("SELECT COUNT(1) FROM foobar WHERE NAME = 'mary'").collect().toSet === Set(Row(1)))
}
test("SELECT * WHERE (quoted strings)") {
@@ -1841,8 +1841,8 @@
rawPlan.execute().count()
}
- assert(getRowCount(df1) == df3.count)
- assert(getRowCount(df2) < df3.count)
+ assert(getRowCount(df1) == df3.count())
+ assert(getRowCount(df2) < df3.count())
}
test("SPARK-26383 throw IllegalArgumentException if wrong kind of driver to the given url") {
@@ -1852,7 +1852,7 @@
"dbtable" -> "table",
"driver" -> "org.postgresql.Driver"
)
- spark.read.format("jdbc").options(opts).load
+ spark.read.format("jdbc").options(opts).load()
}.getMessage
assert(e.contains("The driver could not open a JDBC connection. " +
"Check the URL: jdbc:mysql://localhost/db"))
@@ -2056,7 +2056,7 @@
test("SPARK-41990: Filter with composite name") {
val df = sql("SELECT * FROM composite_name WHERE `last name` = 'smith'")
- assert(df.collect.toSet === Set(Row("smith", 1)))
+ assert(df.collect().toSet === Set(Row("smith", 1)))
}
test("SPARK-44866: SnowflakeDialect BOOLEAN type mapping") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
index b913a39..42cfe7c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
@@ -280,7 +280,7 @@
.options(Map("url" -> url, "dbtable" -> "TEST.SAVETEST"))
.save()
- assert(2 === sqlContext.read.jdbc(url, "TEST.SAVETEST", new Properties).count)
+ assert(2 === sqlContext.read.jdbc(url, "TEST.SAVETEST", new Properties).count())
assert(
2 === sqlContext.read.jdbc(url, "TEST.SAVETEST", new Properties).collect()(0).length)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala
index d675503..c5c56f0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala
@@ -258,7 +258,7 @@
|INSERT INTO TABLE t1 VALUES(2.28, cast("2021-08-08" as date))
|""".stripMargin)
val df = spark.sql("select sum(id) from t1 where id is not null")
- assert(df.count == 1)
+ assert(df.count() == 1)
checkDisableBucketedScan(query = "SELECT SUM(id) FROM t1 WHERE id is not null",
expectedNumScanWithAutoScanEnabled = 1, expectedNumScanWithAutoScanDisabled = 1)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
index eac7be7..baffc50 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
@@ -52,7 +52,7 @@
override def schema: StructType = userSpecifiedSchema
override def insert(input: DataFrame, overwrite: Boolean): Unit = {
- input.collect
+ input.collect()
}
}
@@ -1112,11 +1112,11 @@
withTable("t1", "t2") {
sql("create table t1(j int, s bigint default 42, x bigint default 43) using parquet")
if (useDataFrames) {
- Seq((1, 42, 43)).toDF.write.insertInto("t1")
- Seq((2, 42, 43)).toDF.write.insertInto("t1")
- Seq((3, 42, 43)).toDF.write.insertInto("t1")
- Seq((4, 44, 43)).toDF.write.insertInto("t1")
- Seq((5, 44, 43)).toDF.write.insertInto("t1")
+ Seq((1, 42, 43)).toDF().write.insertInto("t1")
+ Seq((2, 42, 43)).toDF().write.insertInto("t1")
+ Seq((3, 42, 43)).toDF().write.insertInto("t1")
+ Seq((4, 44, 43)).toDF().write.insertInto("t1")
+ Seq((5, 44, 43)).toDF().write.insertInto("t1")
} else {
sql("insert into t1(j) values(1)")
sql("insert into t1(j, s) values(2, default)")
@@ -1769,7 +1769,7 @@
}
def withTableT(f: => Unit): Unit = {
sql(s"create table t(a string, i int) using $dataSource")
- insertIntoT
+ insertIntoT()
withTable("t") { f }
}
// Positive tests:
@@ -1791,7 +1791,7 @@
withTableT {
sql("alter table t add column (s string default concat('abc', 'def'))")
if (config.useDataFrames) {
- Seq((null, null, null)).toDF.write.insertInto("t")
+ Seq((null, null, null)).toDF().write.insertInto("t")
} else {
sql("insert into t values(null, null, null)")
}
@@ -1915,7 +1915,7 @@
sql(s"create table t(a string, i int default 42) using parquet")
checkError(
exception = intercept[AnalysisException] {
- Seq("xyz").toDF.select("value", "default").write.insertInto("t")
+ Seq("xyz").toDF().select("value", "default").write.insertInto("t")
},
errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
parameters = Map("objectName" -> "`default`", "proposal" -> "`value`"))
@@ -1984,7 +1984,7 @@
withTable("t") {
sql(s"create table t(i boolean) using ${config.dataSource}")
if (config.useDataFrames) {
- Seq(false).toDF.write.insertInto("t")
+ Seq(false).toDF().write.insertInto("t")
} else {
sql("insert into t select false")
}
@@ -2003,7 +2003,7 @@
withTable("t") {
sql(s"create table t(i boolean) using ${config.dataSource}")
if (config.useDataFrames) {
- Seq((false)).toDF.write.insertInto("t")
+ Seq((false)).toDF().write.insertInto("t")
} else {
sql("insert into t select false")
}
@@ -2041,7 +2041,7 @@
withTable("t") {
sql(s"create table t(i boolean) using ${config.dataSource}")
if (config.useDataFrames) {
- Seq((false)).toDF.write.insertInto("t")
+ Seq((false)).toDF().write.insertInto("t")
} else {
sql("insert into t select false")
}
@@ -2061,7 +2061,7 @@
withTable("t") {
sql(s"create table t(i boolean) using ${config.dataSource}")
if (config.useDataFrames) {
- Seq((false)).toDF.write.insertInto("t")
+ Seq((false)).toDF().write.insertInto("t")
} else {
sql("insert into t select false")
}
@@ -2099,7 +2099,7 @@
withTable("t") {
sql(s"create table t(i boolean) using ${config.dataSource}")
if (config.useDataFrames) {
- Seq((false)).toDF.write.insertInto("t")
+ Seq((false)).toDF().write.insertInto("t")
} else {
sql("insert into t select false")
}
@@ -2125,7 +2125,7 @@
sql("insert into t select 1, default")
sql("alter table t alter column s drop default")
if (config.useDataFrames) {
- Seq((2, null)).toDF.write.insertInto("t")
+ Seq((2, null)).toDF().write.insertInto("t")
} else {
sql("insert into t select 2, default")
}
@@ -2173,7 +2173,7 @@
withTable("t") {
sql(s"create table t(i boolean) using ${config.dataSource}")
if (config.useDataFrames) {
- Seq((false)).toDF.write.insertInto("t")
+ Seq((false)).toDF().write.insertInto("t")
} else {
sql("insert into t select false")
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
index 0b076e0..36ee322 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
@@ -911,7 +911,7 @@
(MONTHS_PER_YEAR * DAYS_PER_MONTH + 2 * DAYS_PER_MONTH) * MILLIS_PER_DAY)
).foreach { case (delayThresholdVariants, expectedMs) =>
delayThresholdVariants.foreach { case delayThreshold =>
- val df = MemoryStream[Int].toDF
+ val df = MemoryStream[Int].toDF()
.withColumn("eventTime", timestamp_seconds($"value"))
.withWatermark("eventTime", delayThreshold)
val eventTimeAttr = df.queryExecution.analyzed.output.find(a => a.name == "eventTime")
@@ -932,7 +932,7 @@
"interval '1 2:3:4' day to hour",
"interval '1 2' year to month").foreach { delayThreshold =>
intercept[AnalysisException] {
- val df = MemoryStream[Int].toDF
+ val df = MemoryStream[Int].toDF()
.withColumn("eventTime", timestamp_seconds($"value"))
.withWatermark("eventTime", delayThreshold)
}
@@ -944,10 +944,10 @@
private def dfWithMultipleWatermarks(
input1: MemoryStream[Int],
input2: MemoryStream[Int]): Dataset[_] = {
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.withColumn("eventTime", timestamp_seconds($"value"))
.withWatermark("eventTime", "10 seconds")
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.withColumn("eventTime", timestamp_seconds($"value"))
.withWatermark("eventTime", "15 seconds")
df1.union(df2).select($"eventTime".cast("int"))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
index b40ab4c..f03f737 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
@@ -212,7 +212,7 @@
// with aggregations using event time windows and watermark, which allows
// aggregation + append mode.
val inputData = MemoryStream[Long]
- val inputDF = inputData.toDF.toDF("time")
+ val inputDF = inputData.toDF().toDF("time")
val outputDf = inputDF
.selectExpr("timestamp_seconds(time) AS timestamp")
.withWatermark("timestamp", "10 seconds")
@@ -588,7 +588,7 @@
"fs.file.impl.disable.cache" -> "true") {
withTempDir { tempDir =>
val path = new File(tempDir, "text").getCanonicalPath
- Seq("foo").toDF.write.format("text").save(path)
+ Seq("foo").toDF().write.format("text").save(path)
spark.read.format("text").load(path)
}
}
@@ -600,7 +600,7 @@
"fs.file.impl.disable.cache" -> "true") {
withTempDir { tempDir =>
val path = new File(tempDir, "text").getCanonicalPath
- Seq("foo").toDF.write.format("text").save(path)
+ Seq("foo").toDF().write.format("text").save(path)
spark.read.format("text").load(path + "/*")
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
index 684447e..d6867da 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
@@ -1304,7 +1304,7 @@
try {
assert(q.awaitTermination(streamingTimeout.toMillis))
assert(q.recentProgress.count(_.numInputRows != 0) == 1) // only one trigger was run
- checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 3).map(_.toString).toDF)
+ checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 3).map(_.toString).toDF())
} finally {
q.stop()
}
@@ -1317,7 +1317,7 @@
try {
assert(q2.awaitTermination(streamingTimeout.toMillis))
assert(q2.recentProgress.count(_.numInputRows != 0) == 1) // only one trigger was run
- checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 5).map(_.toString).toDF)
+ checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 5).map(_.toString).toDF())
} finally {
q2.stop()
}
@@ -2410,7 +2410,7 @@
}
object ExistsThrowsExceptionFileSystem {
- val scheme = s"FileStreamSourceSuite${math.abs(Random.nextInt)}fs"
+ val scheme = s"FileStreamSourceSuite${math.abs(Random.nextInt())}fs"
}
class CountListingLocalFileSystem extends RawLocalFileSystem {
@@ -2428,7 +2428,7 @@
}
object CountListingLocalFileSystem {
- val scheme = s"CountListingLocalFileSystem${math.abs(Random.nextInt)}fs"
+ val scheme = s"CountListingLocalFileSystem${math.abs(Random.nextInt())}fs"
val pathToNumListStatusCalled = new mutable.HashMap[String, AtomicLong]
def resetCount(): Unit = pathToNumListStatusCalled.clear()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala
index 20fb17f..006cc1c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala
@@ -224,7 +224,7 @@
name = "pandas_grouped_map_with_state", pythonScript = pythonScript)
val inputData = MemoryStream[String]
- val inputDataDS = inputData.toDS
+ val inputDataDS = inputData.toDS()
val outputStructType = StructType(
Seq(
StructField("key", StringType),
@@ -308,7 +308,7 @@
val clock = new StreamManualClock
val inputData = MemoryStream[String]
- val inputDataDS = inputData.toDS
+ val inputDataDS = inputData.toDS()
val outputStructType = StructType(
Seq(
StructField("key", StringType),
@@ -415,7 +415,7 @@
val inputData = MemoryStream[(String, Int)]
val inputDataDF =
- inputData.toDF.select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime"))
+ inputData.toDF().select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime"))
val outputStructType = StructType(
Seq(
StructField("key", StringType),
@@ -497,7 +497,7 @@
val clock = new StreamManualClock
val inputData = MemoryStream[(String, Long)]
val inputDataDF = inputData
- .toDF.toDF("key", "time")
+ .toDF().toDF("key", "time")
.selectExpr("key", "timestamp_seconds(time) as timestamp")
val outputStructType = StructType(
Seq(
@@ -762,7 +762,7 @@
val clock = new StreamManualClock
val inputData = MemoryStream[String]
- val inputDataDS = inputData.toDS
+ val inputDataDS = inputData.toDS()
.withColumnRenamed("value", "key1")
// the type of columns with string literal will be non-nullable
.withColumn("key2", lit("__FAKE__"))
@@ -847,7 +847,7 @@
val clock = new StreamManualClock
val inputData = MemoryStream[String]
// schema: val1, key2, val2, key1, val3
- val inputDataDS = inputData.toDS
+ val inputDataDS = inputData.toDS()
.withColumnRenamed("value", "val1")
.withColumn("key2", $"val1")
// the type of columns with string literal will be non-nullable
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
index a3774bf..b35e9961 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
@@ -452,10 +452,10 @@
if (state.exists) throw new IllegalArgumentException("state.exists should be false")
Iterator((key, values.size))
}
- val df = Seq("a", "a", "b").toDS
+ val df = Seq("a", "a", "b").toDS()
.groupByKey(x => x)
- .flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc).toDF
- checkAnswer(df, Seq(("a", 2), ("b", 1)).toDF)
+ .flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc).toDF()
+ checkAnswer(df, Seq(("a", 2), ("b", 1)).toDF())
}
testWithAllStateVersions("flatMapGroupsWithState - streaming with processing time timeout") {
@@ -528,7 +528,7 @@
testWithAllStateVersions("flatMapGroupsWithState - streaming w/ event time timeout + watermark") {
val inputData = MemoryStream[(String, Int)]
val result =
- inputData.toDS
+ inputData.toDS()
.select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime"))
.withWatermark("eventTime", "10 seconds")
.as[(String, Long)]
@@ -583,7 +583,7 @@
test("flatMapGroupsWithState - recovery from checkpoint uses state format version 1") {
val inputData = MemoryStream[(String, Int)]
val result =
- inputData.toDS
+ inputData.toDS()
.select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime"))
.withWatermark("eventTime", "10 seconds")
.as[(String, Long)]
@@ -697,14 +697,14 @@
spark.createDataset(Seq("a", "a", "b"))
.groupByKey(x => x)
.mapGroupsWithState(EventTimeTimeout)(stateFunc)
- .toDF,
- spark.createDataset(Seq(("a", 2), ("b", 1))).toDF)
+ .toDF(),
+ spark.createDataset(Seq(("a", 2), ("b", 1))).toDF())
}
test("SPARK-35896: metrics in StateOperatorProgress are output correctly") {
val inputData = MemoryStream[(String, Int)]
val result =
- inputData.toDS
+ inputData.toDS()
.select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime"))
.withWatermark("eventTime", "10 seconds")
.as[(String, Long)]
@@ -825,7 +825,7 @@
test("output partitioning is unknown") {
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => key
val inputData = MemoryStream[String]
- val result = inputData.toDS.groupByKey(x => x).mapGroupsWithState(stateFunc)
+ val result = inputData.toDS().groupByKey(x => x).mapGroupsWithState(stateFunc)
testStream(result, Update)(
AddData(inputData, "a"),
CheckNewAnswer("a"),
@@ -1042,7 +1042,7 @@
val stateFormatVersion = spark.conf.get(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION)
val emptyRdd = spark.sparkContext.emptyRDD[InternalRow]
MemoryStream[Int]
- .toDS
+ .toDS()
.groupByKey(x => x)
.flatMapGroupsWithState[Int, Int](Append, timeoutConf = timeoutType)(func)
.logicalPlan.collectFirst {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala
index 93dac34..d795e7a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala
@@ -42,8 +42,8 @@
assert(testState.get === prodState.get)
assert(testState.getTimeoutTimestampMs === prodState.getTimeoutTimestampMs)
assert(testState.hasTimedOut === prodState.hasTimedOut)
- assert(testState.getCurrentProcessingTimeMs === prodState.getCurrentProcessingTimeMs)
- assert(testState.getCurrentWatermarkMs === prodState.getCurrentWatermarkMs)
+ assert(testState.getCurrentProcessingTimeMs() === prodState.getCurrentProcessingTimeMs())
+ assert(testState.getCurrentWatermarkMs() === prodState.getCurrentWatermarkMs())
testState.update(6)
prodState.update(6)
@@ -403,7 +403,8 @@
// Tests for getCurrentProcessingTimeMs in batch queries
val currentTime = System.currentTimeMillis()
- assert(batchState(timeoutConf, watermarkPresent).getCurrentProcessingTimeMs >= currentTime)
+ assert(
+ batchState(timeoutConf, watermarkPresent).getCurrentProcessingTimeMs() >= currentTime)
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala
index fb5445a..405c0bb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala
@@ -836,12 +836,12 @@
}
val input1 = MemoryStream[(String, Timestamp)]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.selectExpr("_1 as leftId", "_2 as leftEventTime")
.withWatermark("leftEventTime", "5 minutes")
val input2 = MemoryStream[(String, Timestamp)]
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.selectExpr("_1 as rightId", "_2 as rightEventTime")
.withWatermark("rightEventTime", "10 minutes")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
index c97979a..e54ce64 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
@@ -94,7 +94,7 @@
}
assert(streamingRelation.nonEmpty, "cannot find StreamingRelation")
assert(
- streamingRelation.head.computeStats.sizeInBytes ==
+ streamingRelation.head.computeStats().sizeInBytes ==
spark.sessionState.conf.defaultSizeInBytes)
}
}
@@ -105,7 +105,8 @@
}
assert(streamingRelation.nonEmpty, "cannot find StreamingRelationV2")
assert(
- streamingRelation.head.computeStats.sizeInBytes == spark.sessionState.conf.defaultSizeInBytes)
+ streamingRelation.head.computeStats().sizeInBytes ==
+ spark.sessionState.conf.defaultSizeInBytes)
}
test("StreamingExecutionRelation.computeStats") {
@@ -113,7 +114,8 @@
val executionRelation = StreamingExecutionRelation(
memoryStream, toAttributes(memoryStream.encoder.schema), None)(
memoryStream.sqlContext.sparkSession)
- assert(executionRelation.computeStats.sizeInBytes == spark.sessionState.conf.defaultSizeInBytes)
+ assert(executionRelation.computeStats().sizeInBytes ==
+ spark.sessionState.conf.defaultSizeInBytes)
}
test("explain join with a normal source") {
@@ -146,7 +148,7 @@
val smallTable3 = Seq((1, "one"), (2, "two"), (4, "four")).toDF("number", "word")
// Join the input stream with a table.
- val df = MemoryStream[Int].toDF
+ val df = MemoryStream[Int].toDF()
val joined = df.join(smallTable, smallTable("number") === $"value")
.join(smallTable2, smallTable2("number") === $"value")
.join(smallTable3, smallTable3("number") === $"value")
@@ -279,7 +281,7 @@
// Running streaming plan as a batch query
assertError("start" :: Nil) {
- streamInput.toDS.map { i => i }.count()
+ streamInput.toDS().map { i => i }.count()
}
// Running non-streaming plan with as a streaming query
@@ -290,7 +292,7 @@
// Running streaming plan that cannot be incrementalized
assertError("not supported" :: "streaming" :: Nil) {
- val ds = streamInput.toDS.map { i => i }.sort()
+ val ds = streamInput.toDS().map { i => i }.sort()
testStream(ds)()
}
}
@@ -647,7 +649,7 @@
test("SPARK-19065: dropDuplicates should not create expressions using the same id") {
withTempPath { testPath =>
val data = Seq((1, 2), (2, 3), (3, 4))
- data.toDS.write.mode("overwrite").json(testPath.getCanonicalPath)
+ data.toDS().write.mode("overwrite").json(testPath.getCanonicalPath)
val schema = spark.read.json(testPath.getCanonicalPath).schema
val query = spark
.readStream
@@ -875,7 +877,7 @@
withTempDir { dir =>
val checkpointLocation = dir.getCanonicalPath
assert(!checkpointLocation.startsWith("file:/"))
- val query = MemoryStream[Int].toDF
+ val query = MemoryStream[Int].toDF()
.writeStream
.option("checkpointLocation", checkpointLocation)
.format("console")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
index c690885..5c3d8d8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
@@ -264,7 +264,7 @@
test("SPARK-19841: watermarkPredicate should filter based on keys") {
val input = MemoryStream[(Int, Int)]
- val df = input.toDS.toDF("time", "id")
+ val df = input.toDS().toDF("time", "id")
.withColumn("time", timestamp_seconds($"time"))
.withWatermark("time", "1 second")
.dropDuplicates("id", "time") // Change the column positions
@@ -283,7 +283,7 @@
test("SPARK-21546: dropDuplicates should ignore watermark when it's not a key") {
val input = MemoryStream[(Int, Int)]
- val df = input.toDS.toDF("id", "time")
+ val df = input.toDS().toDF("id", "time")
.withColumn("time", timestamp_seconds($"time"))
.withWatermark("time", "1 second")
.dropDuplicates("id")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
index 3e1bc57..a380451 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
@@ -55,7 +55,7 @@
protected def setupStream(prefix: String, multiplier: Int): (MemoryStream[Int], DataFrame) = {
val input = MemoryStream[Int]
- val df = input.toDF
+ val df = input.toDF()
.select(
$"value" as "key",
timestamp_seconds($"value") as s"${prefix}Time",
@@ -163,12 +163,12 @@
val leftInput = MemoryStream[(Int, Int)]
val rightInput = MemoryStream[(Int, Int)]
- val df1 = leftInput.toDF.toDF("leftKey", "time")
+ val df1 = leftInput.toDF().toDF("leftKey", "time")
.select($"leftKey", timestamp_seconds($"time") as "leftTime",
($"leftKey" * 2) as "leftValue")
.withWatermark("leftTime", watermark)
- val df2 = rightInput.toDF.toDF("rightKey", "time")
+ val df2 = rightInput.toDF().toDF("rightKey", "time")
.select($"rightKey", timestamp_seconds($"time") as "rightTime",
($"rightKey" * 3) as "rightValue")
.withWatermark("rightTime", watermark)
@@ -232,8 +232,8 @@
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]
- val df1 = input1.toDF.select($"value" as "key", ($"value" * 2) as "leftValue")
- val df2 = input2.toDF.select($"value" as "key", ($"value" * 3) as "rightValue")
+ val df1 = input1.toDF().select($"value" as "key", ($"value" * 2) as "leftValue")
+ val df2 = input2.toDF().select($"value" as "key", ($"value" * 3) as "rightValue")
val joined = df1.join(df2, "key")
testStream(joined)(
@@ -261,12 +261,12 @@
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.select($"value" as "key", timestamp_seconds($"value") as "timestamp",
($"value" * 2) as "leftValue")
.select($"key", window($"timestamp", "10 second"), $"leftValue")
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.select($"value" as "key", timestamp_seconds($"value") as "timestamp",
($"value" * 3) as "rightValue")
.select($"key", window($"timestamp", "10 second"), $"rightValue")
@@ -302,13 +302,13 @@
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.select($"value" as "key", timestamp_seconds($"value") as "timestamp",
($"value" * 2) as "leftValue")
.withWatermark("timestamp", "10 seconds")
.select($"key", window($"timestamp", "10 second"), $"leftValue")
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.select($"value" as "key", timestamp_seconds($"value") as "timestamp",
($"value" * 3) as "rightValue")
.select($"key", window($"timestamp", "10 second"), $"rightValue")
@@ -353,12 +353,12 @@
val leftInput = MemoryStream[(Int, Int)]
val rightInput = MemoryStream[(Int, Int)]
- val df1 = leftInput.toDF.toDF("leftKey", "time")
+ val df1 = leftInput.toDF().toDF("leftKey", "time")
.select($"leftKey", timestamp_seconds($"time") as "leftTime",
($"leftKey" * 2) as "leftValue")
.withWatermark("leftTime", "10 seconds")
- val df2 = rightInput.toDF.toDF("rightKey", "time")
+ val df2 = rightInput.toDF().toDF("rightKey", "time")
.select($"rightKey", timestamp_seconds($"time") as "rightTime",
($"rightKey" * 3) as "rightValue")
.withWatermark("rightTime", "10 seconds")
@@ -413,12 +413,12 @@
val leftInput = MemoryStream[(Int, Int)]
val rightInput = MemoryStream[(Int, Int)]
- val df1 = leftInput.toDF.toDF("leftKey", "time")
+ val df1 = leftInput.toDF().toDF("leftKey", "time")
.select($"leftKey", timestamp_seconds($"time") as "leftTime",
($"leftKey" * 2) as "leftValue")
.withWatermark("leftTime", "20 seconds")
- val df2 = rightInput.toDF.toDF("rightKey", "time")
+ val df2 = rightInput.toDF().toDF("rightKey", "time")
.select($"rightKey", timestamp_seconds($"time") as "rightTime",
($"rightKey" * 3) as "rightValue")
.withWatermark("rightTime", "30 seconds")
@@ -497,9 +497,9 @@
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.select($"value" as "leftKey", ($"value" * 2) as "leftValue")
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.select($"value" as "rightKey", ($"value" * 3) as "rightValue")
val joined = df1.join(df2, expr("leftKey < rightKey"))
val e = intercept[Exception] {
@@ -512,7 +512,7 @@
test("stream stream self join") {
val input = MemoryStream[Int]
- val df = input.toDF
+ val df = input.toDF()
val join =
df.select($"value" % 5 as "key", $"value").join(
df.select($"value" % 5 as "key", $"value"), "key")
@@ -537,7 +537,8 @@
withTempDir { tempDir =>
val queryId = UUID.randomUUID
val opId = 0
- val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextFloat.toString).toString
+ val path =
+ Utils.createDirectory(tempDir.getAbsolutePath, Random.nextFloat().toString).toString
val stateInfo = StatefulOperatorStateInfo(path, queryId, opId, 0L, 5)
implicit val sqlContext = spark.sqlContext
@@ -579,10 +580,10 @@
val input2 = MemoryStream[Int]
val input3 = MemoryStream[Int]
- val df1 = input1.toDF.select($"value" as "leftKey", ($"value" * 2) as "leftValue")
- val df2 = input2.toDF
+ val df1 = input1.toDF().select($"value" as "leftKey", ($"value" * 2) as "leftValue")
+ val df2 = input2.toDF()
.select($"value" as "middleKey", ($"value" * 3) as "middleValue")
- val df3 = input3.toDF
+ val df3 = input3.toDF()
.select($"value" as "rightKey", ($"value" * 5) as "rightValue")
val joined = df1.join(df2, expr("leftKey = middleKey")).join(df3, expr("rightKey = middleKey"))
@@ -598,9 +599,9 @@
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.select($"value" as Symbol("a"), $"value" * 2 as Symbol("b"))
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.select($"value" as Symbol("a"), $"value" * 2 as Symbol("b"))
.repartition($"b")
val joined = df1.join(df2, Seq("a", "b")).select($"a")
@@ -691,13 +692,13 @@
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.select($"value" as "key", timestamp_seconds($"value") as "timestamp",
($"value" * 2) as "leftValue")
.withWatermark("timestamp", "10 seconds")
.select($"key", window($"timestamp", "10 second"), $"leftValue")
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.select($"value" as "key", timestamp_seconds($"value") as "timestamp",
($"value" * 3) as "rightValue")
.select($"key", window($"timestamp", "10 second"), $"rightValue")
@@ -1151,7 +1152,7 @@
def constructUnionDf(desiredPartitionsForInput1: Int)
: (MemoryStream[Int], MemoryStream[Int], MemoryStream[Int], DataFrame) = {
val input1 = MemoryStream[Int](desiredPartitionsForInput1)
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.select(
$"value" as "key",
$"value" as "leftValue",
@@ -1206,12 +1207,12 @@
test("SPARK-32148 stream-stream join regression on Spark 3.0.0") {
val input1 = MemoryStream[(Timestamp, String, String)]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.selectExpr("_1 as eventTime", "_2 as id", "_3 as comment")
.withWatermark(s"eventTime", "2 minutes")
val input2 = MemoryStream[(Timestamp, String, String)]
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.selectExpr("_1 as eventTime", "_2 as id", "_3 as name")
.withWatermark(s"eventTime", "4 minutes")
@@ -1366,12 +1367,12 @@
SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName) {
val input1 = MemoryStream[(Timestamp, String, String)]
- val df1 = input1.toDF
+ val df1 = input1.toDF()
.selectExpr("_1 as eventTime", "_2 as id", "_3 as comment")
.withWatermark("eventTime", "0 second")
val input2 = MemoryStream[(Timestamp, String, String)]
- val df2 = input2.toDF
+ val df2 = input2.toDF()
.selectExpr("_1 as eventTime", "_2 as id", "_3 as comment")
.withWatermark("eventTime", "0 second")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala
index a0f3853..5bf346d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala
@@ -66,7 +66,7 @@
extends AssertOnQuery(q => {
eventually(Timeout(streamingTimeout)) {
if (q.exception.isEmpty) {
- assert(clock.isStreamWaitingAt(clock.getTimeMillis))
+ assert(clock.isStreamWaitingAt(clock.getTimeMillis()))
}
}
if (q.exception.isDefined) {
@@ -210,7 +210,7 @@
test("adding and removing listener") {
def isListenerActive(listener: EventCollector): Boolean = {
listener.reset()
- testStream(MemoryStream[Int].toDS)(
+ testStream(MemoryStream[Int].toDS())(
StartStream(),
StopStream
)
@@ -241,7 +241,7 @@
for (i <- 1 to 50) {
listener.reset()
require(listener.startEvent === null)
- testStream(MemoryStream[Int].toDS)(
+ testStream(MemoryStream[Int].toDS())(
StartStream(),
Assert(listener.startEvent !== null, "onQueryStarted not called before query returned"),
StopStream,
@@ -335,7 +335,7 @@
actions += AssertOnQuery { q =>
q.recentProgress.size > 1 && q.recentProgress.size <= 11
}
- testStream(input.toDS)(actions.toSeq: _*)
+ testStream(input.toDS())(actions.toSeq: _*)
spark.sparkContext.listenerBus.waitUntilEmpty()
// 11 is the max value of the possible numbers of events.
assert(numIdleEvent > 1 && numIdleEvent <= 11)
@@ -355,7 +355,7 @@
collector1.reset()
collector2.reset()
val mem = MemoryStream[Int](implicitly[Encoder[Int]], session.sqlContext)
- testStream(mem.toDS)(
+ testStream(mem.toDS())(
AddData(mem, 1, 2, 3),
CheckAnswer(1, 2, 3)
)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala
index 153ba7a..305d9ca 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala
@@ -38,7 +38,7 @@
.set("spark.bbb", "bbb")
test("test if the configured query listener is loaded") {
- testStream(MemoryStream[Int].toDS)(
+ testStream(MemoryStream[Int].toDS())(
StartStream(),
StopStream
)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala
index 7deb0c6..754f552 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala
@@ -115,11 +115,11 @@
// Terminate a query asynchronously with exception and see awaitAnyTermination throws
// the exception
val q2 = stopRandomQueryAsync(100.milliseconds, withError = true)
- testAwaitAnyTermination(ExpectException[SparkException])
+ testAwaitAnyTermination(ExpectException[SparkException]())
require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned
// All subsequent calls to awaitAnyTermination should throw the exception
- testAwaitAnyTermination(ExpectException[SparkException])
+ testAwaitAnyTermination(ExpectException[SparkException]())
// Resetting termination should make awaitAnyTermination() blocking again
spark.streams.resetTerminated()
@@ -133,7 +133,7 @@
val q4 = stopRandomQueryAsync(10.milliseconds, withError = true)
eventually(Timeout(streamingTimeout)) { require(!q4.isActive) }
// After q4 terminates with exception, awaitAnyTerm should start throwing exception
- testAwaitAnyTermination(ExpectException[SparkException])
+ testAwaitAnyTermination(ExpectException[SparkException]())
}
}
@@ -181,14 +181,14 @@
// throws the exception
val q2 = stopRandomQueryAsync(100.milliseconds, withError = true)
testAwaitAnyTermination(
- ExpectException[SparkException],
+ ExpectException[SparkException](),
awaitTimeout = 4.seconds,
testBehaviorFor = 6.seconds)
require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned
// All subsequent calls to awaitAnyTermination should throw the exception
testAwaitAnyTermination(
- ExpectException[SparkException],
+ ExpectException[SparkException](),
awaitTimeout = 2.seconds,
testBehaviorFor = 4.seconds)
@@ -208,7 +208,7 @@
// `StreamingQueryManager` has already received the error.
q3.stop()
testAwaitAnyTermination(
- ExpectException[SparkException],
+ ExpectException[SparkException](),
awaitTimeout = 100.milliseconds,
testBehaviorFor = 4.seconds)
@@ -228,7 +228,7 @@
// `StreamingQueryManager` has already received the error.
q5.stop()
// After q5 terminates with exception, awaitAnyTerm should start throwing exception
- testAwaitAnyTermination(ExpectException[SparkException], awaitTimeout = 2.seconds)
+ testAwaitAnyTermination(ExpectException[SparkException](), awaitTimeout = 2.seconds)
}
}
@@ -409,7 +409,7 @@
datasets.zipWithIndex.map { case (ds, i) =>
var query: StreamingQuery = null
try {
- val df = ds.toDF
+ val df = ds.toDF()
val metadataRoot =
Utils.createTempDir(namePrefix = "streaming.checkpoint").getCanonicalPath
query =
@@ -480,7 +480,7 @@
private def makeDataset: (MemoryStream[Int], Dataset[Int]) = {
val inputData = MemoryStream[Int]
- val mapped = inputData.toDS.map(6 / _)
+ val mapped = inputData.toDS().map(6 / _)
(inputData, mapped)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 9444db3..1e0fa5b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -69,7 +69,7 @@
test("name unique in active queries") {
withTempDir { dir =>
def startQuery(name: Option[String]): StreamingQuery = {
- val writer = MemoryStream[Int].toDS.writeStream
+ val writer = MemoryStream[Int].toDS().writeStream
name.foreach(writer.queryName)
writer
.foreach(new TestForeachWriter)
@@ -164,9 +164,9 @@
AddData(inputData, 0),
ExpectFailure[SparkException](),
AssertOnQuery(_.isActive === false),
- TestAwaitTermination(ExpectException[SparkException]),
- TestAwaitTermination(ExpectException[SparkException], timeoutMs = 2000),
- TestAwaitTermination(ExpectException[SparkException], timeoutMs = 10),
+ TestAwaitTermination(ExpectException[SparkException]()),
+ TestAwaitTermination(ExpectException[SparkException](), timeoutMs = 2000),
+ TestAwaitTermination(ExpectException[SparkException](), timeoutMs = 10),
AssertOnQuery(q => {
q.exception.get.startOffset ===
q.committedOffsets.toOffsetSeq(Seq(inputData), OffsetSeqMetadata()).toString &&
@@ -245,7 +245,7 @@
}
// query execution should take 350 ms the first time it is called
- val mapped = inputData.toDS.coalesce(1).as[Long].map { x =>
+ val mapped = inputData.toDS().coalesce(1).as[Long].map { x =>
clock.waitTillTime(1500) // this will only wait the first time when clock < 1500
10 / x
}.agg(count("*")).as[Long]
@@ -431,7 +431,7 @@
assert(spark.conf.get(SQLConf.STREAMING_METRICS_ENABLED.key).toBoolean === false)
withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "false") {
- testStream(inputData.toDF)(
+ testStream(inputData.toDF())(
AssertOnQuery { q => !isMetricsRegistered(q) },
StopStream,
AssertOnQuery { q => !isMetricsRegistered(q) }
@@ -440,7 +440,7 @@
// Registered when enabled
withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "true") {
- testStream(inputData.toDF)(
+ testStream(inputData.toDF())(
AssertOnQuery { q => isMetricsRegistered(q) },
StopStream,
AssertOnQuery { q => !isMetricsRegistered(q) }
@@ -485,7 +485,7 @@
}
test("input row calculation with same V1 source used twice in self-join") {
- val streamingTriggerDF = spark.createDataset(1 to 10).toDF
+ val streamingTriggerDF = spark.createDataset(1 to 10).toDF()
val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value")
val progress = getStreamingQuery(streamingInputDF.join(streamingInputDF, "value"))
@@ -496,7 +496,7 @@
}
test("input row calculation with mixed batch and streaming V1 sources") {
- val streamingTriggerDF = spark.createDataset(1 to 10).toDF
+ val streamingTriggerDF = spark.createDataset(1 to 10).toDF()
val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value")
val staticInputDF = spark.createDataFrame(Seq(1 -> "1", 2 -> "2")).toDF("value", "anotherValue")
@@ -511,7 +511,7 @@
test("input row calculation with trigger input DF having multiple leaves in V1 source") {
val streamingTriggerDF =
- spark.createDataset(1 to 5).toDF.union(spark.createDataset(6 to 10).toDF)
+ spark.createDataset(1 to 5).toDF().union(spark.createDataset(6 to 10).toDF())
require(streamingTriggerDF.logicalPlan.collectLeaves().size > 1)
val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF)
@@ -826,14 +826,14 @@
}
val input = MemoryStream[Int] :: MemoryStream[Int] :: MemoryStream[Int] :: Nil
- val q1 = startQuery(input(0).toDS, "stream_serializable_test_1")
- val q2 = startQuery(input(1).toDS.map { i =>
+ val q1 = startQuery(input(0).toDS(), "stream_serializable_test_1")
+ val q2 = startQuery(input(1).toDS().map { i =>
// Emulate that `StreamingQuery` get captured with normal usage unintentionally.
// It should not fail the query.
val q = q1
i
}, "stream_serializable_test_2")
- val q3 = startQuery(input(2).toDS.map { i =>
+ val q3 = startQuery(input(2).toDS().map { i =>
// Emulate that `StreamingQuery` is used in executors. We should fail the query with a clear
// error message.
q1.explain()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
index d03e8bc..4cdfc83 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
@@ -601,7 +601,7 @@
assert(!userCheckpointPath.exists(), s"$userCheckpointPath should not exist")
withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) {
val queryName = "test_query"
- val ds = MemoryStream[Int].toDS
+ val ds = MemoryStream[Int].toDS()
ds.writeStream
.format("memory")
.queryName(queryName)
@@ -623,7 +623,7 @@
withTempDir { checkpointPath =>
withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) {
val queryName = "test_query"
- val ds = MemoryStream[Int].toDS
+ val ds = MemoryStream[Int].toDS()
ds.writeStream.format("memory").queryName(queryName).start().stop()
// Should use query name to create a folder in `checkpointPath`
val queryCheckpointDir = new File(checkpointPath, queryName)
@@ -639,7 +639,7 @@
import testImplicits._
withTempDir { checkpointPath =>
withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) {
- val ds = MemoryStream[Int].toDS
+ val ds = MemoryStream[Int].toDS()
ds.writeStream.format("console").start().stop()
// Should create a random folder in `checkpointPath`
assert(
@@ -655,7 +655,7 @@
withTempDir { checkpointPath =>
withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath,
SQLConf.FORCE_DELETE_TEMP_CHECKPOINT_LOCATION.key -> "true") {
- val ds = MemoryStream[Int].toDS
+ val ds = MemoryStream[Int].toDS()
val query = ds.writeStream.format("console").start()
assert(checkpointPath.exists())
query.stop()
@@ -666,7 +666,7 @@
test("temp checkpoint dir should be deleted if a query is stopped without errors") {
import testImplicits._
- val query = MemoryStream[Int].toDS.writeStream.format("console").start()
+ val query = MemoryStream[Int].toDS().writeStream.format("console").start()
query.processAllAvailable()
val checkpointDir = new Path(
query.asInstanceOf[StreamingQueryWrapper].streamingQuery.resolvedCheckpointRoot)
@@ -690,7 +690,7 @@
private def testTempCheckpointWithFailedQuery(checkpointMustBeDeleted: Boolean): Unit = {
import testImplicits._
val input = MemoryStream[Int]
- val query = input.toDS.map(_ / 0).writeStream.format("console").start()
+ val query = input.toDS().map(_ / 0).writeStream.format("console").start()
val checkpointDir = new Path(
query.asInstanceOf[StreamingQueryWrapper].streamingQuery.resolvedCheckpointRoot)
val fs = checkpointDir.getFileSystem(spark.sessionState.newHadoopConf())
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala
index ca36528..f27fc88 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala
@@ -136,7 +136,7 @@
// Check the query statistics page
val activeQueryLink =
- findAll(cssSelector("""#active-table td a""")).flatMap(_.attribute("href")).next
+ findAll(cssSelector("""#active-table td a""")).flatMap(_.attribute("href")).next()
go to activeQueryLink
findAll(cssSelector("h3"))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
index 66e07e6..4b97a0c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
@@ -252,7 +252,7 @@
}
test("SPARK-32364: later option should override earlier options for save()") {
- Seq(1).toDF.write
+ Seq(1).toDF().write
.format("org.apache.spark.sql.test")
.option("paTh", "1")
.option("PATH", "2")
@@ -264,7 +264,7 @@
withClue("SPARK-32516: legacy path option behavior") {
withSQLConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key -> "true") {
- Seq(1).toDF.write
+ Seq(1).toDF().write
.format("org.apache.spark.sql.test")
.option("paTh", "1")
.option("PATH", "2")
@@ -277,7 +277,7 @@
}
test("pass partitionBy as options") {
- Seq(1).toDF.write
+ Seq(1).toDF().write
.format("org.apache.spark.sql.test")
.partitionBy("col1", "col2")
.save()
@@ -1212,7 +1212,7 @@
withSQLConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key -> "true") {
withTempDir { dir =>
val path = dir.getCanonicalPath
- Seq(1).toDF.write.mode("overwrite").parquet(path)
+ Seq(1).toDF().write.mode("overwrite").parquet(path)
// When there is one path parameter to load(), "path" option is overwritten.
checkAnswer(spark.read.format("parquet").option("path", path).load(path), Row(1))
@@ -1239,7 +1239,7 @@
"Either remove the path option, or call save() without the parameter"))
}
- val df = Seq(1).toDF
+ val df = Seq(1).toDF()
val path = "tmp"
verifyLoadFails(df.write.option("path", path).parquet(path))
verifyLoadFails(df.write.option("path", path).parquet(""))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala
index dfa0348..d4c3d7a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala
@@ -26,20 +26,20 @@
class GenericFlatSpecSuite extends AnyFlatSpec with SharedSparkSessionBase {
import testImplicits._
- private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS
+ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS()
"A Simple Dataset" should "have the specified number of elements" in {
- assert(8 === ds.count)
+ assert(8 === ds.count())
}
it should "have the specified number of unique elements" in {
- assert(8 === ds.distinct.count)
+ assert(8 === ds.distinct().count())
}
it should "have the specified number of elements in each column" in {
- assert(8 === ds.select("_1").count)
- assert(8 === ds.select("_2").count)
+ assert(8 === ds.select("_1").count())
+ assert(8 === ds.select("_2").count())
}
it should "have the correct number of distinct elements in each column" in {
- assert(8 === ds.select("_1").distinct.count)
- assert(4 === ds.select("_2").distinct.count)
+ assert(8 === ds.select("_1").distinct().count())
+ assert(4 === ds.select("_2").distinct().count())
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala
index d15e5c4..d1b8b2f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala
@@ -26,22 +26,22 @@
class GenericFunSpecSuite extends AnyFunSpec with SharedSparkSessionBase {
import testImplicits._
- private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS
+ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS()
describe("Simple Dataset") {
it("should have the specified number of elements") {
- assert(8 === ds.count)
+ assert(8 === ds.count())
}
it("should have the specified number of unique elements") {
- assert(8 === ds.distinct.count)
+ assert(8 === ds.distinct().count())
}
it("should have the specified number of elements in each column") {
- assert(8 === ds.select("_1").count)
- assert(8 === ds.select("_2").count)
+ assert(8 === ds.select("_1").count())
+ assert(8 === ds.select("_2").count())
}
it("should have the correct number of distinct elements in each column") {
- assert(8 === ds.select("_1").distinct.count)
- assert(4 === ds.select("_2").distinct.count)
+ assert(8 === ds.select("_1").distinct().count())
+ assert(4 === ds.select("_2").distinct().count())
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala
index e693dc9..c0725f7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala
@@ -26,25 +26,25 @@
class GenericWordSpecSuite extends AnyWordSpec with SharedSparkSessionBase {
import testImplicits._
- private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS
+ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS()
"A Simple Dataset" when {
"looked at as complete rows" should {
"have the specified number of elements" in {
- assert(8 === ds.count)
+ assert(8 === ds.count())
}
"have the specified number of unique elements" in {
- assert(8 === ds.distinct.count)
+ assert(8 === ds.distinct().count())
}
}
"refined to specific columns" should {
"have the specified number of elements in each column" in {
- assert(8 === ds.select("_1").count)
- assert(8 === ds.select("_2").count)
+ assert(8 === ds.select("_1").count())
+ assert(8 === ds.select("_2").count())
}
"have the correct number of distinct elements in each column" in {
- assert(8 === ds.select("_1").distinct.count)
- assert(4 === ds.select("_2").distinct.count)
+ assert(8 === ds.select("_1").distinct().count())
+ assert(4 === ds.select("_2").distinct().count())
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
index f046daa..808f783 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
@@ -112,7 +112,7 @@
val df = Seq(1).toDF("i")
- df.foreach(r => f)
+ df.foreach(r => f())
df.reduce((x, y) => x)
sparkContext.listenerBus.waitUntilEmpty()
diff --git a/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala
index c63c748..deb2c8f 100644
--- a/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala
@@ -128,7 +128,7 @@
.filter(_.getAs[Int]("age") <= 30)
.sort()
- ds.toDF
+ ds.toDF()
}
test("SPARK-44334: Status of a failed DDL/DML with no jobs should be FAILED") {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 155861a..e70d058 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -230,7 +230,7 @@
result.queryExecution.toString())
iter = if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) {
new IterableFetchIterator[Row](new Iterable[Row] {
- override def iterator: Iterator[Row] = result.toLocalIterator.asScala
+ override def iterator: Iterator[Row] = result.toLocalIterator().asScala
})
} else {
new ArrayFetchIterator[Row](result.collect())
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index ceba74e..5b76cd6 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -218,7 +218,7 @@
// Execute -i init files (always in silent mode)
cli.processInitFiles(sessionState)
- cli.printMasterAndAppId
+ cli.printMasterAndAppId()
if (sessionState.execString != null) {
exit(cli.processLine(sessionState.execString))
@@ -238,7 +238,7 @@
reader.setBellEnabled(false)
reader.setExpandEvents(false)
// reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true)))
- getCommandCompleter.foreach(reader.addCompleter)
+ getCommandCompleter().foreach(reader.addCompleter)
val historyDirectory = System.getProperty("user.home")
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index db72218..ddfe01b 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -1189,7 +1189,7 @@
protected val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator)
protected val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator)
- val localhost = Utils.localCanonicalHostName
+ val localhost = Utils.localCanonicalHostName()
private var listeningPort: Int = _
protected def serverPort: Int = listeningPort
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
index a3f1a06..94f1e53 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
@@ -36,7 +36,7 @@
def checkResult(rs: ResultSet, dbNames: Seq[String]): Unit = {
val expected = dbNames.iterator
while(rs.next() || expected.hasNext) {
- assert(rs.getString("TABLE_SCHEM") === expected.next)
+ assert(rs.getString("TABLE_SCHEM") === expected.next())
assert(rs.getString("TABLE_CATALOG").isEmpty)
}
// Make sure there are no more elements
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala
index eef2ae1..811d186 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala
@@ -65,7 +65,7 @@
qe.assertCommandExecuted()
} else {
tableDesc.storage.locationUri.foreach { p =>
- DataWritingCommand.assertEmptyRootPath(p, mode, sparkSession.sessionState.newHadoopConf)
+ DataWritingCommand.assertEmptyRootPath(p, mode, sparkSession.sessionState.newHadoopConf())
}
// TODO ideally, we should get the output data ready first and then
// add the relation into catalog, just in case of failure occurs while data
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
index 400befb..8ff209d 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
@@ -163,7 +163,7 @@
private def testFieldQuery(fields: Seq[String]): Unit = {
if (!fields.isEmpty) {
val query = s"SELECT * FROM ${TEST_TABLE_NAME} WHERE ${Random.shuffle(fields).head} >= 0"
- assert(spark.sql(query).count == NUM_RECORDS)
+ assert(spark.sql(query).count() == NUM_RECORDS)
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index 62b2d84..05d2ca1 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -2851,7 +2851,7 @@
.select("data_type")
// check if the last access time doesn't have the default date of year
// 1970 as its a wrong access time
- assert((desc.first.toString.contains("UNKNOWN")))
+ assert((desc.first().toString.contains("UNKNOWN")))
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index b2a6e45..a658067 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -880,7 +880,7 @@
"""CREATE TEMPORARY FUNCTION udtf_count2 AS
|'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
""".stripMargin)
- assert(sql("DESCRIBE FUNCTION udtf_count2").count > 1)
+ assert(sql("DESCRIBE FUNCTION udtf_count2").count() > 1)
sql("DROP TEMPORARY FUNCTION udtf_count2")
}
@@ -946,7 +946,7 @@
assert(sql(s"list archive ${zipFile.getAbsolutePath}").count() === 1)
assert(sql(s"list archives ${zipFile.getAbsolutePath} nonexistence").count() === 1)
assert(sql(s"list archives ${zipFile.getAbsolutePath} " +
- s"${jarFile.getAbsolutePath}").count === 2)
+ s"${jarFile.getAbsolutePath}").count() === 2)
}
}
@@ -990,7 +990,7 @@
filter(_.getString(0).contains(s"${xzFile.getAbsolutePath}")).count() > 0)
assert(sql(s"list archive ${bz2File.getAbsolutePath}").count() === 1)
assert(sql(s"list archives ${bz2File.getAbsolutePath} " +
- s"${xzFile.getAbsolutePath}").count === 2)
+ s"${xzFile.getAbsolutePath}").count() === 2)
}
}
@@ -1010,7 +1010,7 @@
sql(s"""ADD FILES "${file3.getAbsolutePath}" ${file4.getAbsoluteFile}""")
val listFiles = sql(s"LIST FILES ${file1.getAbsolutePath} " +
s"'${file2.getAbsolutePath}' '${file3.getAbsolutePath}' ${file4.getAbsolutePath}")
- assert(listFiles.count === 4)
+ assert(listFiles.count() === 4)
assert(listFiles.filter(_.getString(0).contains(file1.getName)).count() === 1)
assert(listFiles.filter(
_.getString(0).contains(file2.getName.replace(" ", "%20"))).count() === 1)
@@ -1046,7 +1046,7 @@
sql(s"ADD JARS ${jarFile3.getAbsolutePath} '${jarFile4.getAbsoluteFile}'")
val listFiles = sql(s"LIST JARS '${jarFile1.getAbsolutePath}' " +
s"${jarFile2.getAbsolutePath} ${jarFile3.getAbsolutePath} '${jarFile4.getAbsoluteFile}'")
- assert(listFiles.count === 4)
+ assert(listFiles.count() === 4)
assert(listFiles.filter(
_.getString(0).contains(jarFile1.getName.replace(" ", "%20"))).count() === 1)
assert(listFiles.filter(_.getString(0).contains(jarFile2.getName)).count() === 1)
@@ -1082,7 +1082,7 @@
sql(s"ADD ARCHIVES ${jarFile3.getAbsolutePath} '${jarFile4.getAbsoluteFile}'")
val listFiles = sql(s"LIST ARCHIVES ${jarFile1.getAbsolutePath} " +
s"'${jarFile2.getAbsolutePath}' ${jarFile3.getAbsolutePath} '${jarFile4.getAbsolutePath}'")
- assert(listFiles.count === 4)
+ assert(listFiles.count() === 4)
assert(listFiles.filter(_.getString(0).contains(jarFile1.getName)).count() === 1)
assert(listFiles.filter(
_.getString(0).contains(jarFile2.getName.replace(" ", "%20"))).count() === 1)
@@ -1108,7 +1108,7 @@
val listFiles = sql("LIST FILES " +
s"""'${file1.getAbsolutePath}' ${file2.getAbsolutePath} "${file3.getAbsolutePath}"""")
- assert(listFiles.count === 3)
+ assert(listFiles.count() === 3)
assert(listFiles.filter(_.getString(0).contains(file1.getName)).count() === 1)
assert(listFiles.filter(_.getString(0).contains(file2.getName)).count() === 1)
assert(listFiles.filter(
@@ -1134,7 +1134,7 @@
val listArchives = sql(s"LIST ARCHIVES '${jarFile1.getAbsolutePath}' " +
s"""${jarFile2.getAbsolutePath} "${jarFile3.getAbsolutePath}"""")
- assert(listArchives.count === 3)
+ assert(listArchives.count() === 3)
assert(listArchives.filter(_.getString(0).contains(jarFile1.getName)).count() === 1)
assert(listArchives.filter(_.getString(0).contains(jarFile2.getName)).count() === 1)
assert(listArchives.filter(
@@ -1159,7 +1159,7 @@
sql(s"ADD JAR '${jarFile6.getAbsolutePath}'")
val listJars = sql(s"LIST JARS '${jarFile4.getAbsolutePath}' " +
s"""${jarFile5.getAbsolutePath} "${jarFile6.getAbsolutePath}"""")
- assert(listJars.count === 3)
+ assert(listJars.count() === 3)
assert(listJars.filter(_.getString(0).contains(jarFile4.getName)).count() === 1)
assert(listJars.filter(_.getString(0).contains(jarFile5.getName)).count() === 1)
assert(listJars.filter(
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
index 1eff35c..d188e2d 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
@@ -493,7 +493,7 @@
sql("SELECT input_file_name() as file FROM external_t5").head().getString(0)
assert(answer1.contains("data1") || answer1.contains("data2"))
- val count2 = sql("SELECT input_file_name() as file FROM external_t5").distinct().count
+ val count2 = sql("SELECT input_file_name() as file FROM external_t5").distinct().count()
assert(count2 == 2)
sql("DROP TABLE external_t5")
}
@@ -515,7 +515,7 @@
sql("SELECT input_file_name() as file FROM external_parquet").head().getString(0)
assert(answer3.contains("external_parquet"))
- val count3 = sql("SELECT input_file_name() as file FROM external_parquet").distinct().count
+ val count3 = sql("SELECT input_file_name() as file FROM external_parquet").distinct().count()
assert(count3 == 1)
sql("DROP TABLE external_parquet")
}
@@ -527,7 +527,7 @@
sql("SELECT input_file_name() as file FROM parquet_tmp").head().getString(0)
assert(answer4.contains("parquet_tmp"))
- val count4 = sql("SELECT input_file_name() as file FROM parquet_tmp").distinct().count
+ val count4 = sql("SELECT input_file_name() as file FROM parquet_tmp").distinct().count()
assert(count4 == 1)
sql("DROP TABLE parquet_tmp")
}
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 0ede331..d8c28e1 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
@@ -149,8 +149,8 @@
Order(1, "Atlas", "MTB", 434, "2015-01-07", "John D", "Pacifica", "CA", 20151),
Order(11, "Swift", "YFlikr", 137, "2015-01-23", "John D", "Hayward", "CA", 20151))
- orders.toDF.createOrReplaceTempView("orders1")
- orderUpdates.toDF.createOrReplaceTempView("orderupdates1")
+ orders.toDF().createOrReplaceTempView("orders1")
+ orderUpdates.toDF().createOrReplaceTempView("orderupdates1")
withTable("orders", "orderupdates") {
sql(
@@ -356,7 +356,7 @@
test("explode nested Field") {
withTempView("nestedArray") {
- Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.createOrReplaceTempView("nestedArray")
+ Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF().createOrReplaceTempView("nestedArray")
checkAnswer(
sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"),
Row(1) :: Row(2) :: Row(3) :: Nil)
@@ -1410,7 +1410,7 @@
test("run sql directly on files - hive") {
withTempPath(f => {
- spark.range(100).toDF.write.parquet(f.getCanonicalPath)
+ spark.range(100).toDF().write.parquet(f.getCanonicalPath)
checkError(
exception = intercept[AnalysisException] {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala
index 0bd6b14..2eff462 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala
@@ -129,8 +129,8 @@
s1
}
def finish(s: Array[Double]): Array[Double] = s
- def bufferEncoder: Encoder[Array[Double]] = ExpressionEncoder[Array[Double]]
- def outputEncoder: Encoder[Array[Double]] = ExpressionEncoder[Array[Double]]
+ def bufferEncoder: Encoder[Array[Double]] = ExpressionEncoder[Array[Double]]()
+ def outputEncoder: Encoder[Array[Double]] = ExpressionEncoder[Array[Double]]()
}
abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala
index 43bcee5..0330ce5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala
@@ -88,7 +88,7 @@
withTempPath { dir =>
withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
@@ -117,7 +117,7 @@
withTempPath { dir =>
withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(
dir,
@@ -148,7 +148,7 @@
withTempPath { dir =>
withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
import spark.implicits._
- spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
+ spark.range(values).map(_ => Random.nextLong()).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
@@ -272,7 +272,7 @@
import spark.implicits._
val middle = width / 2
val selectExpr = (1 to width).map(i => s"value as c$i")
- spark.range(values).map(_ => Random.nextLong).toDF()
+ spark.range(values).map(_ => Random.nextLong()).toDF()
.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT * FROM t1"))
@@ -304,7 +304,7 @@
import spark.implicits._
val selectExprCore = (1 to width).map(i => s"'f$i', value").mkString(",")
val selectExpr = Seq(s"named_struct($selectExprCore) as c1")
- spark.range(values).map(_ => Random.nextLong).toDF()
+ spark.range(values).map(_ => Random.nextLong()).toDF()
.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT * FROM t1"))
@@ -343,7 +343,7 @@
.map(_ => s"$structExpr").mkString(",")
val selectExpr = Seq(s"array($arrayExprElements) as c1")
print(s"select expression is $selectExpr\n")
- spark.range(values).map(_ => Random.nextLong).toDF()
+ spark.range(values).map(_ => Random.nextLong()).toDF()
.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
prepareTable(dir, spark.sql("SELECT * FROM t1"))
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala
index 3f5be78..e1bd6df 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala
@@ -32,7 +32,7 @@
}
def < (that: Interval): Boolean = {
- if (this.duration != that.duration) {
+ if (this.duration() != that.duration()) {
throw new Exception("Comparing two intervals with different durations [" + this + ", "
+ that + "]")
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/State.scala b/streaming/src/main/scala/org/apache/spark/streaming/State.scala
index c4cd1a9..fbfb910 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/State.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/State.scala
@@ -120,10 +120,10 @@
/**
* Get the state as a `scala.Option`. It will be `Some(state)` if it exists, otherwise `None`.
*/
- @inline final def getOption(): Option[S] = if (exists) Some(get()) else None
+ @inline final def getOption(): Option[S] = if (exists()) Some(get()) else None
@inline final override def toString(): String = {
- getOption.map { _.toString }.getOrElse("<state not set>")
+ getOption().map { _.toString }.getOrElse("<state not set>")
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
index 9d3facc..883d56c 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
@@ -88,7 +88,7 @@
def receive(): Unit = {
try {
val iterator = bytesToObjects(socket.getInputStream())
- while(!isStopped && iterator.hasNext) {
+ while(!isStopped() && iterator.hasNext) {
store(iterator.next())
}
if (!isStopped()) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala
index 662312b..097eb9d 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala
@@ -55,10 +55,10 @@
dataIterator.foreach { case (key, value) =>
wrappedState.wrap(newStateMap.get(key))
val returned = mappingFunction(batchTime, key, Some(value), wrappedState)
- if (wrappedState.isRemoved) {
+ if (wrappedState.isRemoved()) {
newStateMap.remove(key)
- } else if (wrappedState.isUpdated
- || (wrappedState.exists && timeoutThresholdTime.isDefined)) {
+ } else if (wrappedState.isUpdated()
+ || (wrappedState.exists() && timeoutThresholdTime.isDefined)) {
newStateMap.put(key, wrappedState.get(), batchTime.milliseconds)
}
mappedData ++= returned
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala
index 7a561ec..7087f16 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala
@@ -82,7 +82,7 @@
val countIterator = new CountingIterator(iterator)
val putResult = blockManager.putIterator(blockId, countIterator, storageLevel,
tellMaster = true)
- numRecords = countIterator.count
+ numRecords = countIterator.count()
putResult
case ByteBufferBlock(byteBuffer) =>
blockManager.putBytes(
@@ -178,7 +178,7 @@
case IteratorBlock(iterator) =>
val countIterator = new CountingIterator(iterator)
val serializedBlock = serializerManager.dataSerialize(blockId, countIterator)
- numRecords = countIterator.count
+ numRecords = countIterator.count()
serializedBlock
case ByteBufferBlock(byteBuffer) =>
new ChunkedByteBuffer(byteBuffer.duplicate())
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala
index 1037950..5aa2a9d 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala
@@ -128,7 +128,7 @@
logDebug(s"Executors (${allExecIds.size}) = ${allExecIds}")
if (allExecIds.nonEmpty && allExecIds.size > minNumExecutors) {
- val execIdsWithReceivers = receiverTracker.allocatedExecutors.values.flatten.toSeq
+ val execIdsWithReceivers = receiverTracker.allocatedExecutors().values.flatten.toSeq
logInfo(s"Executors with receivers (${execIdsWithReceivers.size}): ${execIdsWithReceivers}")
val removableExecIds = allExecIds.diff(execIdsWithReceivers)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
index 13d10ec..29bccc7 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@ -78,7 +78,7 @@
// attach rate controllers of input streams to receive batch completion updates
for {
- inputDStream <- ssc.graph.getInputStreams
+ inputDStream <- ssc.graph.getInputStreams()
rateController <- inputDStream.rateController
} ssc.addStreamingListener(rateController)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala
index 07c9e63..c3f2a04 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala
@@ -59,7 +59,7 @@
import FileBasedWriteAheadLog._
private val pastLogs = new ArrayBuffer[LogInfo]
- private val callerName = getCallerName
+ private val callerName = getCallerName()
private val threadpoolName = {
"WriteAheadLogManager" + callerName.map(c => s" for $c").getOrElse("")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
index 3ffb2c1..3ebe39f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
@@ -26,7 +26,7 @@
private val thread = new Thread("RecurringTimer - " + name) {
setDaemon(true)
- override def run(): Unit = { loop }
+ override def run(): Unit = { loop() }
}
@volatile private var prevTime = -1L
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 4429cde..9060828 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -775,7 +775,7 @@
// verify that the latest input blocks are present but the earliest blocks have been removed
assert(latestBlockRdd.isValid)
- assert(latestBlockRdd.collect != null)
+ assert(latestBlockRdd.collect() != null)
assert(!earliestBlockRdd.isValid)
earliestBlockRdd.blockIds.foreach { blockId =>
assert(!ssc.sparkContext.env.blockManager.master.contains(blockId))
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index 6757cb7..9760682 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -168,7 +168,7 @@
eventually(timeout(10.seconds)) {
val checkpointFilesOfLatestTime = Checkpoint.getCheckpointFiles(checkpointDir).filter {
- _.getName.contains(clock.getTimeMillis.toString)
+ _.getName.contains(clock.getTimeMillis().toString)
}
// Checkpoint files are written twice for every batch interval. So assert that both
// are written to make sure that both of them have been written.
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 2a22433..ca8307b 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -322,7 +322,7 @@
// set up the network stream using the test receiver
withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc =>
val networkStream = ssc.receiverStream[Int](testReceiver)
- val countStream = networkStream.count
+ val countStream = networkStream.count()
val outputStream = new TestOutputStream(countStream, outputQueue)
outputStream.register()
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala
index 09048c8..e88aab2b 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala
@@ -66,17 +66,17 @@
shouldBeTimingOut: Boolean = false
): Unit = {
if (expectedData.isDefined) {
- assert(state.exists)
+ assert(state.exists())
assert(state.get() === expectedData.get)
assert(state.getOption() === expectedData)
- assert(state.getOption.getOrElse(-1) === expectedData.get)
+ assert(state.getOption().getOrElse(-1) === expectedData.get)
} else {
- assert(!state.exists)
+ assert(!state.exists())
intercept[NoSuchElementException] {
state.get()
}
assert(state.getOption() === None)
- assert(state.getOption.getOrElse(-1) === -1)
+ assert(state.getOption().getOrElse(-1) === -1)
}
assert(state.isTimingOut() === shouldBeTimingOut)
@@ -161,7 +161,7 @@
// state maintains running count, and updated count is returned
val mappingFunc = (key: String, value: Option[Int], state: State[Int]) => {
- val sum = value.getOrElse(0) + state.getOption.getOrElse(0)
+ val sum = value.getOrElse(0) + state.getOption().getOrElse(0)
state.update(sum)
sum
}
@@ -206,7 +206,7 @@
// state maintains running count, key string doubled and returned
val mappingFunc = (batchTime: Time, key: String, value: Option[Int], state: State[Int]) => {
- val sum = value.getOrElse(0) + state.getOption.getOrElse(0)
+ val sum = value.getOrElse(0) + state.getOption().getOrElse(0)
state.update(sum)
Some(key * 2)
}
@@ -298,7 +298,7 @@
)
val mappingFunc = (time: Time, key: String, value: Option[Int], state: State[Int]) => {
- val sum = value.getOrElse(0) + state.getOption.getOrElse(0)
+ val sum = value.getOrElse(0) + state.getOption().getOrElse(0)
val output = (key, sum)
state.update(sum)
Some(output)
@@ -336,7 +336,7 @@
)
val mappingFunc = (time: Time, key: String, value: Option[Int], state: State[Int]) => {
- val sum = value.getOrElse(0) + state.getOption.getOrElse(0)
+ val sum = value.getOrElse(0) + state.getOption().getOrElse(0)
val output = (key, sum)
state.update(sum)
None.asInstanceOf[Option[Int]]
@@ -385,7 +385,7 @@
)
val mappingFunc = (time: Time, key: String, value: Option[Int], state: State[Int]) => {
- if (state.exists) {
+ if (state.exists()) {
state.remove()
Some(key)
} else {
@@ -413,7 +413,7 @@
if (value.isDefined) {
state.update(1)
}
- if (state.isTimingOut) {
+ if (state.isTimingOut()) {
Some(key)
} else {
None
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala
index c1af91a..ffaa82d 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala
@@ -333,7 +333,7 @@
try {
// If it is the first killing, then allow the first checkpoint to be created
val minKillWaitTime = if (MasterFailureTest.killCount == 0) 5000 else 2000
- val killWaitTime = minKillWaitTime + math.abs(Random.nextLong % maxKillWaitTime)
+ val killWaitTime = minKillWaitTime + math.abs(Random.nextLong() % maxKillWaitTime)
logInfo("Kill wait time = " + killWaitTime)
Thread.sleep(killWaitTime)
logInfo(
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala
index 1913552..1bf74e6 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala
@@ -361,7 +361,7 @@
}
def dataToByteBuffer(b: Seq[String]) =
- serializerManager.dataSerialize(generateBlockId, b.iterator)
+ serializerManager.dataSerialize(generateBlockId(), b.iterator)
val blocks = data.grouped(10).toSeq
@@ -425,7 +425,7 @@
handler: ReceivedBlockHandler,
block: ReceivedBlock
): (StreamBlockId, ReceivedBlockStoreResult) = {
- val blockId = generateBlockId
+ val blockId = generateBlockId()
val blockStoreResult = handler.storeBlock(blockId, block)
logDebug("Done inserting")
(blockId, blockStoreResult)
@@ -435,7 +435,8 @@
getLogFilesInDirectory(checkpointDirToLogDir(tempDirectory.toString, streamId))
}
- private def generateBlockId(): StreamBlockId = StreamBlockId(streamId, scala.util.Random.nextLong)
+ private def generateBlockId(): StreamBlockId =
+ StreamBlockId(streamId, scala.util.Random.nextLong())
}
class ReceivedBlockHandlerSuite extends BaseReceivedBlockHandlerSuite(false)
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala
index ada6a9a..e221a3a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala
@@ -385,7 +385,7 @@
/** Generate blocks infos using random ids */
def generateBlockInfos(blockCount: Int = 5): Seq[ReceivedBlockInfo] = {
List.fill(blockCount)(ReceivedBlockInfo(streamId, Some(0L), None,
- BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt)), Some(0L))))
+ BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt())), Some(0L))))
}
/**
@@ -416,7 +416,7 @@
* Get all the data written in the given write ahead log files. By default, it will read all
* files in the test log directory.
*/
- def getWrittenLogData(logFiles: Seq[String] = getWriteAheadLogFiles)
+ def getWrittenLogData(logFiles: Seq[String] = getWriteAheadLogFiles())
: Seq[ReceivedBlockTrackerLogEvent] = {
logFiles.flatMap {
file => new FileBasedWriteAheadLogReader(file, hadoopConf).toSeq
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
index b07fd73..afe6c73 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
@@ -74,8 +74,8 @@
// Verify that receiver was started
assert(receiver.callsRecorder.calls === Seq("onStart"))
- assert(executor.isReceiverStarted)
- assert(receiver.isStarted)
+ assert(executor.isReceiverStarted())
+ assert(receiver.isStarted())
assert(!receiver.isStopped())
assert(receiver.otherThread.isAlive)
eventually(timeout(100.milliseconds), interval(10.milliseconds)) {
@@ -111,8 +111,8 @@
receiver.restart("restarting", null, 100)
eventually(timeout(10.seconds), interval(10.milliseconds)) {
// below verification ensures for now receiver is already restarted
- assert(receiver.isStarted)
- assert(!receiver.isStopped)
+ assert(receiver.isStarted())
+ assert(!receiver.isStopped())
assert(receiver.receiving)
// both receiver supervisor and receiver should be stopped first, and started
@@ -127,7 +127,7 @@
// Verify that stopping actually stops the thread
failAfter(1.second) {
receiver.stop("test")
- assert(receiver.isStopped)
+ assert(receiver.isStopped())
assert(!receiver.otherThread.isAlive)
// The thread that started the executor should complete
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
index 29eb1db..c704a41 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
@@ -177,7 +177,7 @@
ssc.start()
}
assert(ssc.getState() === StreamingContextState.STOPPED)
- assert(ssc.scheduler.isStarted === false)
+ assert(ssc.scheduler.isStarted() === false)
}
test("start should set local properties of streaming jobs correctly") {
@@ -600,7 +600,7 @@
newContextCreated = true
val newSsc = new StreamingContext(sc, batchDuration)
val input = addInputStream(newSsc)
- input.foreachRDD { rdd => rdd.count }
+ input.foreachRDD { rdd => rdd.count() }
newSsc
}
@@ -641,7 +641,7 @@
// getActiveOrCreate and getActive should return independently created context after activating
testGetActiveOrCreate {
val sc = new SparkContext(conf)
- ssc = creatingFunc(sc) // Create
+ ssc = creatingFunc(sc)() // Create
assert(StreamingContext.getActive().isEmpty,
"new initialized context returned before starting")
ssc.start()
@@ -733,13 +733,13 @@
conf.clone.set("spark.streaming.clock", "org.apache.spark.util.ManualClock"))
ssc = new StreamingContext(sc, Seconds(1))
val input = addInputStream(ssc)
- input.foreachRDD { rdd => rdd.count }
+ input.foreachRDD { rdd => rdd.count() }
ssc.start()
// Creating another streaming context should not create errors
val anotherSsc = new StreamingContext(sc, Seconds(10))
val anotherInput = addInputStream(anotherSsc)
- anotherInput.foreachRDD { rdd => rdd.count }
+ anotherInput.foreachRDD { rdd => rdd.count() }
val exception = intercept[IllegalStateException] {
anotherSsc.start()
@@ -760,7 +760,7 @@
require(ssc.getState() === StreamingContextState.INITIALIZED)
val input = addInputStream(ssc)
val transformed = input.map { x => x}
- transformed.foreachRDD { rdd => rdd.count }
+ transformed.foreachRDD { rdd => rdd.count() }
def testForException(clue: String, expectedErrorMsg: String)(body: => Unit): Unit = {
withClue(clue) {
@@ -927,7 +927,7 @@
val thread = new Thread() {
override def run(): Unit = {
logInfo("Receiving started")
- while (!isStopped) {
+ while (!isStopped()) {
store(TestReceiver.counter.getAndIncrement)
}
logInfo("Receiving stopped at count value of " + TestReceiver.counter.get())
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
index 2ec4b5b..63899f9 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
@@ -117,7 +117,7 @@
test("receiver info reporting") {
ssc = new StreamingContext("local[2]", "test", Milliseconds(1000))
val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver)
- inputStream.foreachRDD(_.count)
+ inputStream.foreachRDD(_.count())
val collector = new ReceiverInfoCollector
ssc.addStreamingListener(collector)
@@ -163,7 +163,7 @@
test("don't call ssc.stop in listener") {
ssc = new StreamingContext("local[2]", "ssc", Milliseconds(1000))
val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver)
- inputStream.foreachRDD(_.count)
+ inputStream.foreachRDD(_.count())
startStreamingContextAndCallStop(ssc)
}
@@ -171,7 +171,7 @@
test("onBatchCompleted with successful batch") {
ssc = new StreamingContext("local[2]", "test", Milliseconds(1000))
val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver)
- inputStream.foreachRDD(_.count)
+ inputStream.foreachRDD(_.count())
val failureReasons = startStreamingContextAndCollectFailureReasons(ssc)
assert(failureReasons != null && failureReasons.isEmpty,
@@ -220,7 +220,7 @@
ssc = new StreamingContext("local[2]", "test", Milliseconds(1000))
ssc.addStreamingListener(streamingListener)
val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver)
- inputStream.foreachRDD(_.count)
+ inputStream.foreachRDD(_.count())
ssc.start()
ssc.stop()
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 55e4a46..dc6ebaf 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -404,7 +404,7 @@
logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput)
// Get the output buffer
- val outputStream = ssc.graph.getOutputStreams.
+ val outputStream = ssc.graph.getOutputStreams().
filter(_.isInstanceOf[TestOutputStreamWithPartitions[_]]).
head.asInstanceOf[TestOutputStreamWithPartitions[V]]
val output = outputStream.output
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
index 979cfde..eab8012 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
@@ -80,7 +80,7 @@
try {
rdd.foreach { _ =>
// Failing the task with id 15 to ensure only one task fails
- if (TaskContext.get.taskAttemptId() % 15 == 0) {
+ if (TaskContext.get().taskAttemptId() % 15 == 0) {
throw new RuntimeException("Oops")
}
}
@@ -97,7 +97,7 @@
val sparkUI = ssc.sparkContext.ui.get
- sparkUI.getDelegatingHandlers.count(_.getContextPath.contains("/streaming")) should be (5)
+ sparkUI.getDelegatingHandlers.count(_.getContextPath().contains("/streaming")) should be (5)
eventually(timeout(10.seconds), interval(50.milliseconds)) {
go to (sparkUI.webUrl.stripSuffix("/"))
@@ -212,7 +212,7 @@
ssc.stop(false)
- sparkUI.getDelegatingHandlers.count(_.getContextPath.contains("/streaming")) should be (0)
+ sparkUI.getDelegatingHandlers.count(_.getContextPath().contains("/streaming")) should be (0)
eventually(timeout(10.seconds), interval(50.milliseconds)) {
go to (sparkUI.webUrl.stripSuffix("/"))
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala
index 1b59a8c..0250ec9 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala
@@ -108,7 +108,7 @@
case Some("get-state") =>
Some(state.getOption().getOrElse(-1))
case Some("update-state") =>
- if (state.exists) state.update(state.get + 1) else state.update(0)
+ if (state.exists()) state.update(state.get() + 1) else state.update(0)
None
case Some("remove-state") =>
removedStates += state.get()
@@ -232,7 +232,7 @@
// else if the data is 2, remove the state if it exists
data match {
case Some(1) =>
- if (state.exists()) { state.update(state.get + 1) }
+ if (state.exists()) { state.update(state.get() + 1) }
else state.update(0)
case Some(2) =>
state.remove()
@@ -303,7 +303,7 @@
def rddCollectFunc(rdd: RDD[MapWithStateRDDRecord[Int, Int, Int]])
: Set[(List[(Int, Int, Long)], List[Int])] = {
rdd.map { record => (record.stateMap.getAll().toList, record.mappedData.toList) }
- .collect.toSet
+ .collect().toSet
}
/** Generate MapWithStateRDD with data RDD having a long lineage */
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala
index c2b0392..7d9dfb1 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala
@@ -61,7 +61,7 @@
withAllocationManager(numReceivers = 1, conf = conf) {
case (receiverTracker, allocationManager) =>
- when(receiverTracker.allocatedExecutors).thenReturn(Map(1 -> Some("1")))
+ when(receiverTracker.allocatedExecutors()).thenReturn(Map(1 -> Some("1")))
/** Add data point for batch processing time and verify executor allocation */
def addBatchProcTimeAndVerifyAllocation(batchProcTimeMs: Double)(body: => Unit): Unit = {
@@ -239,7 +239,7 @@
reset(allocationClient)
when(allocationClient.getExecutorIds()).thenReturn(execIds)
- when(receiverTracker.allocatedExecutors).thenReturn(receiverExecIds)
+ when(receiverTracker.allocatedExecutors()).thenReturn(receiverExecIds)
killExecutor(allocationManager)
if (expectedKilledExec.nonEmpty) {
verify(allocationClient, times(1)).killExecutor(meq(expectedKilledExec.get))
diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
index a46a7fb..07157da 100644
--- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
@@ -126,7 +126,7 @@
.writeAll(previousContents + privateClasses.mkString("\n"))
// scalastyle:off println
println("Created : .generated-mima-class-excludes in current directory.")
- val previousMembersContents = Try(File(".generated-mima-member-excludes").lines)
+ val previousMembersContents = Try(File(".generated-mima-member-excludes").lines())
.getOrElse(Iterator.empty).mkString("\n")
File(".generated-mima-member-excludes").writeAll(previousMembersContents +
privateMembers.mkString("\n"))
@@ -150,7 +150,7 @@
private def getClasses(packageName: String): Set[String] = {
val finder = ClassFinder(maybeOverrideAsmVersion = Some(Opcodes.ASM7))
finder
- .getClasses
+ .getClasses()
.map(_.name)
.filter(_.startsWith(packageName))
.filterNot(shouldExclude)