Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion core/src/test/scala/org/apache/spark/CheckpointSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -500,7 +500,7 @@ class CheckpointSuite extends SparkFunSuite with RDDCheckpointTester with LocalS
}

runTest("CheckpointRDD with zero partitions") { reliableCheckpoint: Boolean =>
val rdd = new BlockRDD[Int](sc, Array[BlockId]())
val rdd = new BlockRDD[Int](sc, Array.empty[BlockId])
assert(rdd.partitions.size === 0)
assert(rdd.isCheckpointed === false)
assert(rdd.isCheckpointedAndMaterialized === false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ class JsonProtocolSuite extends SparkFunSuite with JsonTestUtils {
test("writeMasterState") {
val workers = Array(createWorkerInfo(), createWorkerInfo())
val activeApps = Array(createAppInfo())
val completedApps = Array[ApplicationInfo]()
val completedApps = Array.empty[ApplicationInfo]
val activeDrivers = Array(createDriverInfo())
val completedDrivers = Array(createDriverInfo())
val stateResponse = new MasterStateResponse(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ class SparkSubmitSuite

// scalastyle:off println
test("prints usage on empty input") {
testPrematureExit(Array[String](), "Usage: spark-submit")
testPrematureExit(Array.empty[String], "Usage: spark-submit")
}

test("prints usage with only --help") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ class HistoryServerArgumentsSuite extends SparkFunSuite {
.set("spark.testing", "true")

test("No Arguments Parsing") {
val argStrings = Array[String]()
val argStrings = Array.empty[String]
val hsa = new HistoryServerArguments(conf, argStrings)
assert(conf.get("spark.history.fs.logDirectory") === logDir.getAbsolutePath)
assert(conf.get("spark.history.fs.updateInterval") === "1")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite {
}

test("toArray()") {
val empty = ByteBuffer.wrap(Array[Byte]())
val empty = ByteBuffer.wrap(Array.empty[Byte])
val bytes = ByteBuffer.wrap(Array.tabulate(8)(_.toByte))
val chunkedByteBuffer = new ChunkedByteBuffer(Array(bytes, bytes, empty))
assert(chunkedByteBuffer.toArray === bytes.array() ++ bytes.array())
Expand All @@ -74,7 +74,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite {
}

test("toInputStream()") {
val empty = ByteBuffer.wrap(Array[Byte]())
val empty = ByteBuffer.wrap(Array.empty[Byte])
val bytes1 = ByteBuffer.wrap(Array.tabulate(256)(_.toByte))
val bytes2 = ByteBuffer.wrap(Array.tabulate(128)(_.toByte))
val chunkedByteBuffer = new ChunkedByteBuffer(Array(empty, bytes1, bytes2))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
check(Array("aaa", "bbb", null))
check(Array(true, false, true))
check(Array('a', 'b', 'c'))
check(Array[Int]())
check(Array.empty[Int])
check(Array(Array("1", "2"), Array("1", "2", "3", "4")))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -287,7 +287,7 @@ class MatricesSuite extends SparkMLFunSuite {
val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2))
val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2))
val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2))
val deHorz2 = Matrices.horzcat(Array[Matrix]())
val deHorz2 = Matrices.horzcat(Array.empty[Matrix])

assert(deHorz1.numRows === 3)
assert(spHorz2.numRows === 3)
Expand Down Expand Up @@ -341,7 +341,7 @@ class MatricesSuite extends SparkMLFunSuite {
val deVert1 = Matrices.vertcat(Array(deMat1, deMat3))
val spVert2 = Matrices.vertcat(Array(spMat1, deMat3))
val spVert3 = Matrices.vertcat(Array(deMat1, spMat3))
val deVert2 = Matrices.vertcat(Array[Matrix]())
val deVert2 = Matrices.vertcat(Array.empty[Matrix])

assert(deVert1.numRows === 5)
assert(spVert2.numRows === 5)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,9 @@ class TestingUtilsSuite extends SparkMLFunSuite {
assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01))
assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01))
assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array[Double]()) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array[Double]()) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01)

// Should throw exception with message when test fails.
intercept[TestFailedException](
Expand All @@ -125,7 +125,7 @@ class TestingUtilsSuite extends SparkMLFunSuite {
Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01)

intercept[TestFailedException](
Vectors.dense(Array[Double]()) ~== Vectors.dense(Array(3.135)) relTol 0.01)
Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01)

// Comparing against zero should fail the test and throw exception with message
// saying that the relative error is meaningless in this situation.
Expand All @@ -145,7 +145,7 @@ class TestingUtilsSuite extends SparkMLFunSuite {
assert(Vectors.dense(Array(3.1)) !~==
Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01)

assert(Vectors.dense(Array[Double]()) !~==
assert(Vectors.dense(Array.empty[Double]) !~==
Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01)
}

Expand Down Expand Up @@ -176,14 +176,14 @@ class TestingUtilsSuite extends SparkMLFunSuite {
assert(!(Vectors.dense(Array(3.1)) ~=
Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5))

assert(Vectors.dense(Array[Double]()) !~=
assert(Vectors.dense(Array.empty[Double]) !~=
Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)

assert(!(Vectors.dense(Array[Double]()) ~=
assert(!(Vectors.dense(Array.empty[Double]) ~=
Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5))

assert(Vectors.dense(Array[Double]()) ~=
Vectors.dense(Array[Double]()) absTol 1E-5)
assert(Vectors.dense(Array.empty[Double]) ~=
Vectors.dense(Array.empty[Double]) absTol 1E-5)

// Should throw exception with message when test fails.
intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~==
Expand All @@ -195,7 +195,7 @@ class TestingUtilsSuite extends SparkMLFunSuite {
intercept[TestFailedException](Vectors.dense(Array(3.1)) ~==
Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6)

intercept[TestFailedException](Vectors.dense(Array[Double]()) ~==
intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~==
Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6)

// Comparisons of two sparse vectors
Expand All @@ -214,7 +214,7 @@ class TestingUtilsSuite extends SparkMLFunSuite {
assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~==
Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3)

assert(Vectors.sparse(0, Array[Int](), Array[Double]()) !~==
assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~==
Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3)

// Comparisons of a dense vector and a sparse vector
Expand All @@ -230,14 +230,14 @@ class TestingUtilsSuite extends SparkMLFunSuite {
assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~==
Vectors.dense(Array(3.1)) absTol 1E-6)

assert(Vectors.dense(Array[Double]()) !~==
assert(Vectors.dense(Array.empty[Double]) !~==
Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6)

assert(Vectors.sparse(1, Array(0), Array(3.1)) !~==
Vectors.dense(Array(3.1, 3.2)) absTol 1E-6)

assert(Vectors.dense(Array(3.1)) !~==
Vectors.sparse(0, Array[Int](), Array[Double]()) absTol 1E-6)
Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6)
}

test("Comparing Matrices using absolute error.") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -622,7 +622,7 @@ class LogisticRegression @Since("1.2.0") (
rawCoefficients(coefIndex)
}
} else {
Array[Double]()
Array.empty[Double]
}
val interceptVector = if (interceptsArray.nonEmpty && isMultinomial) {
// The intercepts are never regularized, so we always center the mean.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,8 @@ private[stat] object KolmogorovSmirnovTest extends Logging {
val pResults = partDiffs.foldLeft(initAcc) { case ((pMin, pMax, pCt), (dl, dp)) =>
(math.min(pMin, dl), math.max(pMax, dp), pCt + 1)
}
val results = if (pResults == initAcc) Array[(Double, Double, Double)]() else Array(pResults)
val results =
if (pResults == initAcc) Array.empty[(Double, Double, Double)] else Array(pResults)
results.iterator
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ class MultilayerPerceptronClassifierSuite
test("Input Validation") {
val mlpc = new MultilayerPerceptronClassifier()
intercept[IllegalArgumentException] {
mlpc.setLayers(Array[Int]())
mlpc.setLayers(Array.empty[Int])
}
intercept[IllegalArgumentException] {
mlpc.setLayers(Array[Int](1))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ class MLSerDeSuite extends SparkFunSuite {
assert(matrix === nm)

// Test conversion for empty matrix
val empty = Array[Double]()
val empty = Array.empty[Double]
val emptyMatrix = Matrices.dense(0, 0, empty)
val ne = MLSerDe.loads(MLSerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix]
assert(emptyMatrix == ne)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,10 +154,10 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext {
val featureSamples = Array(0, 0, 0).map(_.toDouble)
val featureSamplesEmpty = Array.empty[Double]
val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
assert(splits === Array[Double]())
assert(splits === Array.empty[Double])
val splitsEmpty =
RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, fakeMetadata, 0)
assert(splitsEmpty === Array[Double]())
assert(splitsEmpty === Array.empty[Double])
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ class PythonMLLibAPISuite extends SparkFunSuite {
assert(matrix === nm)

// Test conversion for empty matrix
val empty = Array[Double]()
val empty = Array.empty[Double]
val emptyMatrix = Matrices.dense(0, 0, empty)
val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix]
assert(emptyMatrix == ne)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext {
Seq(
(Array(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Array(1, 2, 3, 4, 5)),
(Array(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Array(1, 2, 3)),
(Array(1, 2, 3, 4, 5), Array[Int]())
(Array(1, 2, 3, 4, 5), Array.empty[Int])
), 2)
val eps = 1.0E-5

Expand All @@ -55,7 +55,7 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext {
val predictionAndLabels = sc.parallelize(
Seq(
(Array(1, 6, 2), Array(1, 2, 3, 4, 5)),
(Array[Int](), Array(1, 2, 3))
(Array.empty[Int], Array(1, 2, 3))
), 2)
val eps = 1.0E-5

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -289,7 +289,7 @@ class MatricesSuite extends SparkFunSuite {
val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2))
val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2))
val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2))
val deHorz2 = Matrices.horzcat(Array[Matrix]())
val deHorz2 = Matrices.horzcat(Array.empty[Matrix])

assert(deHorz1.numRows === 3)
assert(spHorz2.numRows === 3)
Expand Down Expand Up @@ -343,7 +343,7 @@ class MatricesSuite extends SparkFunSuite {
val deVert1 = Matrices.vertcat(Array(deMat1, deMat3))
val spVert2 = Matrices.vertcat(Array(spMat1, deMat3))
val spVert3 = Matrices.vertcat(Array(deMat1, spMat3))
val deVert2 = Matrices.vertcat(Array[Matrix]())
val deVert2 = Matrices.vertcat(Array.empty[Matrix])

assert(deVert1.numRows === 5)
assert(spVert2.numRows === 5)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,9 @@ class TestingUtilsSuite extends SparkFunSuite {
assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01))
assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01))
assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array[Double]()) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array[Double]()) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01)
assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01)

// Should throw exception with message when test fails.
intercept[TestFailedException](
Expand All @@ -125,7 +125,7 @@ class TestingUtilsSuite extends SparkFunSuite {
Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01)

intercept[TestFailedException](
Vectors.dense(Array[Double]()) ~== Vectors.dense(Array(3.135)) relTol 0.01)
Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01)

// Comparing against zero should fail the test and throw exception with message
// saying that the relative error is meaningless in this situation.
Expand All @@ -145,7 +145,7 @@ class TestingUtilsSuite extends SparkFunSuite {
assert(Vectors.dense(Array(3.1)) !~==
Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01)

assert(Vectors.dense(Array[Double]()) !~==
assert(Vectors.dense(Array.empty[Double]) !~==
Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01)
}

Expand Down Expand Up @@ -176,14 +176,14 @@ class TestingUtilsSuite extends SparkFunSuite {
assert(!(Vectors.dense(Array(3.1)) ~=
Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5))

assert(Vectors.dense(Array[Double]()) !~=
assert(Vectors.dense(Array.empty[Double]) !~=
Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)

assert(!(Vectors.dense(Array[Double]()) ~=
assert(!(Vectors.dense(Array.empty[Double]) ~=
Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5))

assert(Vectors.dense(Array[Double]()) ~=
Vectors.dense(Array[Double]()) absTol 1E-5)
assert(Vectors.dense(Array.empty[Double]) ~=
Vectors.dense(Array.empty[Double]) absTol 1E-5)

// Should throw exception with message when test fails.
intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~==
Expand All @@ -195,7 +195,7 @@ class TestingUtilsSuite extends SparkFunSuite {
intercept[TestFailedException](Vectors.dense(Array(3.1)) ~==
Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6)

intercept[TestFailedException](Vectors.dense(Array[Double]()) ~==
intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~==
Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6)

// Comparisons of two sparse vectors
Expand All @@ -214,7 +214,7 @@ class TestingUtilsSuite extends SparkFunSuite {
assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~==
Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3)

assert(Vectors.sparse(0, Array[Int](), Array[Double]()) !~==
assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~==
Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3)

// Comparisons of a dense vector and a sparse vector
Expand All @@ -230,14 +230,14 @@ class TestingUtilsSuite extends SparkFunSuite {
assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~==
Vectors.dense(Array(3.1)) absTol 1E-6)

assert(Vectors.dense(Array[Double]()) !~==
assert(Vectors.dense(Array.empty[Double]) !~==
Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6)

assert(Vectors.sparse(1, Array(0), Array(3.1)) !~==
Vectors.dense(Array(3.1, 3.2)) absTol 1E-6)

assert(Vectors.dense(Array(3.1)) !~==
Vectors.sparse(0, Array[Int](), Array[Double]()) absTol 1E-6)
Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6)
}

test("Comparing Matrices using absolute error.") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -215,13 +215,13 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(Substring(bytes, 2, 2), Array[Byte](2, 3))
checkEvaluation(Substring(bytes, 3, 2), Array[Byte](3, 4))
checkEvaluation(Substring(bytes, 4, 2), Array[Byte](4))
checkEvaluation(Substring(bytes, 8, 2), Array[Byte]())
checkEvaluation(Substring(bytes, 8, 2), Array.empty[Byte])
checkEvaluation(Substring(bytes, -1, 2), Array[Byte](4))
checkEvaluation(Substring(bytes, -2, 2), Array[Byte](3, 4))
checkEvaluation(Substring(bytes, -3, 2), Array[Byte](2, 3))
checkEvaluation(Substring(bytes, -4, 2), Array[Byte](1, 2))
checkEvaluation(Substring(bytes, -5, 2), Array[Byte](1))
checkEvaluation(Substring(bytes, -8, 2), Array[Byte]())
checkEvaluation(Substring(bytes, -8, 2), Array.empty[Byte])
}

test("string substring_index function") {
Expand Down Expand Up @@ -275,7 +275,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(Base64(UnBase64(a)), "AQIDBA==", create_row("AQIDBA=="))

checkEvaluation(Base64(b), "AQIDBA==", create_row(bytes))
checkEvaluation(Base64(b), "", create_row(Array[Byte]()))
checkEvaluation(Base64(b), "", create_row(Array.empty[Byte]))
checkEvaluation(Base64(b), null, create_row(null))
checkEvaluation(Base64(Literal.create(null, BinaryType)), null, create_row("abdef"))

Expand Down Expand Up @@ -526,13 +526,13 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
// non ascii characters are not allowed in the source code, so we disable the scalastyle.
checkEvaluation(Length(Literal("a花花c")), 4, create_row(string))
// scalastyle:on
checkEvaluation(Length(Literal(bytes)), 5, create_row(Array[Byte]()))
checkEvaluation(Length(Literal(bytes)), 5, create_row(Array.empty[Byte]))

checkEvaluation(Length(a), 5, create_row(string))
checkEvaluation(Length(b), 5, create_row(bytes))

checkEvaluation(Length(a), 0, create_row(""))
checkEvaluation(Length(b), 0, create_row(Array[Byte]()))
checkEvaluation(Length(b), 0, create_row(Array.empty[Byte]))

checkEvaluation(Length(a), null, create_row(null))
checkEvaluation(Length(b), null, create_row(null))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -273,7 +273,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
test("sort_array function") {
val df = Seq(
(Array[Int](2, 1, 3), Array("b", "c", "a")),
(Array[Int](), Array[String]()),
(Array.empty[Int], Array.empty[String]),
(null, null)
).toDF("a", "b")
checkAnswer(
Expand Down