aboutsummaryrefslogtreecommitdiff
path: root/mllib/src
diff options
context:
space:
mode:
authorSean Owen <sowen@cloudera.com>2016-03-03 09:54:09 +0000
committerSean Owen <sowen@cloudera.com>2016-03-03 09:54:09 +0000
commite97fc7f176f8bf501c9b3afd8410014e3b0e1602 (patch)
tree23a11a3646b13195aaf50078a0f35fad96190618 /mllib/src
parent02b7677e9584f5ccd68869abdb0bf980dc847ce1 (diff)
downloadspark-e97fc7f176f8bf501c9b3afd8410014e3b0e1602.tar.gz
spark-e97fc7f176f8bf501c9b3afd8410014e3b0e1602.tar.bz2
spark-e97fc7f176f8bf501c9b3afd8410014e3b0e1602.zip
[SPARK-13423][WIP][CORE][SQL][STREAMING] Static analysis fixes for 2.x
## What changes were proposed in this pull request? Make some cross-cutting code improvements according to static analysis. These are individually up for discussion since they exist in separate commits that can be reverted. The changes are broadly: - Inner class should be static - Mismatched hashCode/equals - Overflow in compareTo - Unchecked warnings - Misuse of assert, vs junit.assert - get(a) + getOrElse(b) -> getOrElse(a,b) - Array/String .size -> .length (occasionally, -> .isEmpty / .nonEmpty) to avoid implicit conversions - Dead code - tailrec - exists(_ == ) -> contains find + nonEmpty -> exists filter + size -> count - reduce(_+_) -> sum map + flatten -> map The most controversial may be .size -> .length simply because of its size. It is intended to avoid implicits that might be expensive in some places. ## How was the this patch tested? Existing Jenkins unit tests. Author: Sean Owen <sowen@cloudera.com> Closes #11292 from srowen/SPARK-13423.
Diffstat (limited to 'mllib/src')
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala14
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala6
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala6
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala12
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala10
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala8
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala4
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java4
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala2
27 files changed, 61 insertions, 55 deletions
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
index 18be5c0701..3b4209bbc4 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
@@ -166,7 +166,7 @@ class MinMaxScalerModel private[ml] (
// 0 in sparse vector will probably be rescaled to non-zero
val values = vector.toArray
- val size = values.size
+ val size = values.length
var i = 0
while (i < size) {
val raw = if (originalRange(i) != 0) (values(i) - minArray(i)) / originalRange(i) else 0.5
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala
index 769f4406e2..d75b3ef420 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala
@@ -166,7 +166,7 @@ object QuantileDiscretizer extends DefaultParamsReadable[QuantileDiscretizer] wi
* needed, and adding a default split value of 0 if no good candidates are found.
*/
private[feature] def getSplits(candidates: Array[Double]): Array[Double] = {
- val effectiveValues = if (candidates.size != 0) {
+ val effectiveValues = if (candidates.nonEmpty) {
if (candidates.head == Double.NegativeInfinity
&& candidates.last == Double.PositiveInfinity) {
candidates.drop(1).dropRight(1)
@@ -181,7 +181,7 @@ object QuantileDiscretizer extends DefaultParamsReadable[QuantileDiscretizer] wi
candidates
}
- if (effectiveValues.size == 0) {
+ if (effectiveValues.isEmpty) {
Array(Double.NegativeInfinity, 0, Double.PositiveInfinity)
} else {
Array(Double.NegativeInfinity) ++ effectiveValues ++ Array(Double.PositiveInfinity)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index ca0ed95a48..cf17689a3a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -1297,7 +1297,7 @@ private[spark] object SerDe extends Serializable {
def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = {
val m: DenseMatrix = obj.asInstanceOf[DenseMatrix]
- val bytes = new Array[Byte](8 * m.values.size)
+ val bytes = new Array[Byte](8 * m.values.length)
val order = ByteOrder.nativeOrder()
val isTransposed = if (m.isTransposed) 1 else 0
ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values)
@@ -1389,7 +1389,7 @@ private[spark] object SerDe extends Serializable {
def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = {
val v: SparseVector = obj.asInstanceOf[SparseVector]
- val n = v.indices.size
+ val n = v.indices.length
val indiceBytes = new Array[Byte](4 * n)
val order = ByteOrder.nativeOrder()
ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala
index 2910c027ae..4308ae04ee 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala
@@ -77,7 +77,7 @@ private[classification] object GLMClassificationModel {
val sqlContext = SQLContext.getOrCreate(sc)
val dataRDD = sqlContext.read.parquet(datapath)
val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1)
- assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath")
+ assert(dataArray.length == 1, s"Unable to load $modelClass data from: $datapath")
val data = dataArray(0)
assert(data.size == 3, s"Unable to load $modelClass data from: $datapath")
val (weights, intercept) = data match {
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 54bf5102cc..f0b9d64fc4 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
@@ -19,6 +19,7 @@ package org.apache.spark.mllib.clustering
import java.util.Random
+import scala.annotation.tailrec
import scala.collection.mutable
import org.apache.spark.Logging
@@ -467,6 +468,7 @@ private[clustering] class ClusteringTreeNode private[clustering] (
* @param cost the cost to the current center
* @return (predicted leaf cluster index, cost)
*/
+ @tailrec
private def predict(pointWithNorm: VectorWithNorm, cost: Double): (Int, Double) = {
if (isLeaf) {
(index, cost)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
index 3b91fe8643..439e4f8672 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
@@ -144,7 +144,7 @@ object KMeansModel extends Loader[KMeansModel] {
val centroids = sqlContext.read.parquet(Loader.dataPath(path))
Loader.checkSchema[Cluster](centroids.schema)
val localCentroids = centroids.rdd.map(Cluster.apply).collect()
- assert(k == localCentroids.size)
+ assert(k == localCentroids.length)
new KMeansModel(localCentroids.sortBy(_.id).map(_.point))
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala
index 3029b15f58..5dde2bdb17 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala
@@ -66,7 +66,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl
*/
@Since("1.1.0")
def confusionMatrix: Matrix = {
- val n = labels.size
+ val n = labels.length
val values = Array.ofDim[Double](n * n)
var i = 0
while (i < n) {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala
index daf6ff4db4..95b2fef27a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala
@@ -58,8 +58,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val accuracy: Double = predictionAndLabels.map { case (predictions, labels) =>
- labels.intersect(predictions).size.toDouble /
- (labels.size + predictions.size - labels.intersect(predictions).size)}.sum / numDocs
+ labels.intersect(predictions).length.toDouble /
+ (labels.length + predictions.length - labels.intersect(predictions).length)}.sum / numDocs
/**
@@ -67,7 +67,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val hammingLoss: Double = predictionAndLabels.map { case (predictions, labels) =>
- labels.size + predictions.size - 2 * labels.intersect(predictions).size
+ labels.length + predictions.length - 2 * labels.intersect(predictions).length
}.sum / (numDocs * numLabels)
/**
@@ -75,8 +75,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val precision: Double = predictionAndLabels.map { case (predictions, labels) =>
- if (predictions.size > 0) {
- predictions.intersect(labels).size.toDouble / predictions.size
+ if (predictions.length > 0) {
+ predictions.intersect(labels).length.toDouble / predictions.length
} else {
0
}
@@ -87,7 +87,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val recall: Double = predictionAndLabels.map { case (predictions, labels) =>
- labels.intersect(predictions).size.toDouble / labels.size
+ labels.intersect(predictions).length.toDouble / labels.length
}.sum / numDocs
/**
@@ -95,7 +95,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val f1Measure: Double = predictionAndLabels.map { case (predictions, labels) =>
- 2.0 * predictions.intersect(labels).size / (predictions.size + labels.size)
+ 2.0 * predictions.intersect(labels).length / (predictions.length + labels.length)
}.sum / numDocs
private lazy val tpPerClass = predictionAndLabels.flatMap { case (predictions, labels) =>
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
index cffa9fba05..9457c6e9e3 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
@@ -88,7 +88,7 @@ private object IDF {
}
doc match {
case SparseVector(size, indices, values) =>
- val nnz = indices.size
+ val nnz = indices.length
var k = 0
while (k < nnz) {
if (values(k) > 0) {
@@ -97,7 +97,7 @@ private object IDF {
k += 1
}
case DenseVector(values) =>
- val n = values.size
+ val n = values.length
var j = 0
while (j < n) {
if (values(j) > 0.0) {
@@ -211,7 +211,7 @@ private object IDFModel {
val n = v.size
v match {
case SparseVector(size, indices, values) =>
- val nnz = indices.size
+ val nnz = indices.length
val newValues = new Array[Double](nnz)
var k = 0
while (k < nnz) {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
index af0c8e1d8a..99fcb36f27 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
@@ -55,7 +55,7 @@ class Normalizer @Since("1.1.0") (p: Double) extends VectorTransformer {
vector match {
case DenseVector(vs) =>
val values = vs.clone()
- val size = values.size
+ val size = values.length
var i = 0
while (i < size) {
values(i) /= norm
@@ -64,7 +64,7 @@ class Normalizer @Since("1.1.0") (p: Double) extends VectorTransformer {
Vectors.dense(values)
case SparseVector(size, ids, vs) =>
val values = vs.clone()
- val nnz = values.size
+ val nnz = values.length
var i = 0
while (i < nnz) {
values(i) /= norm
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
index 6fe573c528..500187ae86 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
@@ -132,7 +132,7 @@ class StandardScalerModel @Since("1.3.0") (
vector match {
case DenseVector(vs) =>
val values = vs.clone()
- val size = values.size
+ val size = values.length
if (withStd) {
var i = 0
while (i < size) {
@@ -153,7 +153,7 @@ class StandardScalerModel @Since("1.3.0") (
vector match {
case DenseVector(vs) =>
val values = vs.clone()
- val size = values.size
+ val size = values.length
var i = 0
while(i < size) {
values(i) *= (if (std(i) != 0.0) 1.0 / std(i) else 0.0)
@@ -164,7 +164,7 @@ class StandardScalerModel @Since("1.3.0") (
// For sparse vector, the `index` array inside sparse vector object will not be changed,
// so we can re-use it to save memory.
val values = vs.clone()
- val nnz = values.size
+ val nnz = values.length
var i = 0
while (i < nnz) {
values(i) *= (if (std(indices(i)) != 0.0) 1.0 / std(indices(i)) else 0.0)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
index 3241ebeb22..b046f684ca 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
@@ -346,9 +346,9 @@ class Word2Vec extends Serializable with Logging {
if (alpha < learningRate * 0.0001) alpha = learningRate * 0.0001
logInfo("wordCount = " + wordCount + ", alpha = " + alpha)
}
- wc += sentence.size
+ wc += sentence.length
var pos = 0
- while (pos < sentence.size) {
+ while (pos < sentence.length) {
val word = sentence(pos)
val b = random.nextInt(window)
// Train Skip-gram
@@ -356,7 +356,7 @@ class Word2Vec extends Serializable with Logging {
while (a < window * 2 + 1 - b) {
if (a != window) {
val c = pos - window + a
- if (c >= 0 && c < sentence.size) {
+ if (c >= 0 && c < sentence.length) {
val lastWord = sentence(c)
val l1 = lastWord * vectorSize
val neu1e = new Array[Float](vectorSize)
@@ -579,7 +579,7 @@ object Word2VecModel extends Loader[Word2VecModel] {
private def buildWordVectors(model: Map[String, Array[Float]]): Array[Float] = {
require(model.nonEmpty, "Word2VecMap should be non-empty")
- val (vectorSize, numWords) = (model.head._2.size, model.size)
+ val (vectorSize, numWords) = (model.head._2.length, model.size)
val wordList = model.keys.toArray
val wordVectors = new Array[Float](vectorSize * numWords)
var i = 0
@@ -615,7 +615,7 @@ object Word2VecModel extends Loader[Word2VecModel] {
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._
- val vectorSize = model.values.head.size
+ val vectorSize = model.values.head.length
val numWords = model.size
val metadata = compact(render(
("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~
@@ -646,7 +646,7 @@ object Word2VecModel extends Loader[Word2VecModel] {
(loadedClassName, loadedVersion) match {
case (classNameV1_0, "1.0") =>
val model = SaveLoadV1_0.load(sc, path)
- val vectorSize = model.getVectors.values.head.size
+ val vectorSize = model.getVectors.values.head.length
val numWords = model.getVectors.size
require(expectedVectorSize == vectorSize,
s"Word2VecModel requires each word to be mapped to a vector of size " +
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
index b35d7217d6..f5b4f2535f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
@@ -232,7 +232,7 @@ class FPGrowth private (
partitioner: Partitioner): Array[Item] = {
data.flatMap { t =>
val uniq = t.toSet
- if (t.size != uniq.size) {
+ if (t.length != uniq.size) {
throw new SparkException(s"Items in a transaction must be unique but got ${t.toSeq}.")
}
t
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
index df9f4ae145..d2687dc11b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
@@ -75,7 +75,7 @@ private[spark] object BLAS extends Serializable with Logging {
val xValues = x.values
val xIndices = x.indices
val yValues = y.values
- val nnz = xIndices.size
+ val nnz = xIndices.length
if (a == 1.0) {
var k = 0
@@ -135,7 +135,7 @@ private[spark] object BLAS extends Serializable with Logging {
val xValues = x.values
val xIndices = x.indices
val yValues = y.values
- val nnz = xIndices.size
+ val nnz = xIndices.length
var sum = 0.0
var k = 0
@@ -154,8 +154,8 @@ private[spark] object BLAS extends Serializable with Logging {
val xIndices = x.indices
val yValues = y.values
val yIndices = y.indices
- val nnzx = xIndices.size
- val nnzy = yIndices.size
+ val nnzx = xIndices.length
+ val nnzy = yIndices.length
var kx = 0
var ky = 0
@@ -188,7 +188,7 @@ private[spark] object BLAS extends Serializable with Logging {
val sxIndices = sx.indices
val sxValues = sx.values
val dyValues = dy.values
- val nnz = sxIndices.size
+ val nnz = sxIndices.length
var i = 0
var k = 0
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala
index ffdcddec11..e4494792bb 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala
@@ -33,7 +33,7 @@ private[spark] object CholeskyDecomposition {
* @return the solution array
*/
def solve(A: Array[Double], bx: Array[Double]): Array[Double] = {
- val k = bx.size
+ val k = bx.length
val info = new intW(0)
lapack.dppsv("U", k, 1, A, bx, k, info)
val code = info.`val`
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
index b08da4fb55..0fdb402fd6 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
@@ -987,7 +987,7 @@ object Matrices {
def horzcat(matrices: Array[Matrix]): Matrix = {
if (matrices.isEmpty) {
return new DenseMatrix(0, 0, Array[Double]())
- } else if (matrices.size == 1) {
+ } else if (matrices.length == 1) {
return matrices(0)
}
val numRows = matrices(0).numRows
@@ -1046,7 +1046,7 @@ object Matrices {
def vertcat(matrices: Array[Matrix]): Matrix = {
if (matrices.isEmpty) {
return new DenseMatrix(0, 0, Array[Double]())
- } else if (matrices.size == 1) {
+ } else if (matrices.length == 1) {
return matrices(0)
}
val numCols = matrices(0).numCols
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala
index 09527dcf5d..ae1faf6a2d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala
@@ -176,7 +176,7 @@ class BlockMatrix @Since("1.3.0") (
val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt
private[mllib] def createPartitioner(): GridPartitioner =
- GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = blocks.partitions.size)
+ GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = blocks.partitions.length)
private lazy val blockInfo = blocks.mapValues(block => (block.numRows, block.numCols)).cache()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
index e8de515211..06b9c4ac67 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
@@ -120,9 +120,9 @@ class IndexedRowMatrix @Since("1.0.0") (
val rowIndex = row.index
row.vector match {
case SparseVector(size, indices, values) =>
- Iterator.tabulate(indices.size)(i => MatrixEntry(rowIndex, indices(i), values(i)))
+ Iterator.tabulate(indices.length)(i => MatrixEntry(rowIndex, indices(i), values(i)))
case DenseVector(values) =>
- Iterator.tabulate(values.size)(i => MatrixEntry(rowIndex, i, values(i)))
+ Iterator.tabulate(values.length)(i => MatrixEntry(rowIndex, i, values(i)))
}
}
new CoordinateMatrix(entries, numRows(), numCols())
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index 3e619c4264..a7a843ab76 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -226,12 +226,12 @@ class ALS private (
val sc = ratings.context
val numUserBlocks = if (this.numUserBlocks == -1) {
- math.max(sc.defaultParallelism, ratings.partitions.size / 2)
+ math.max(sc.defaultParallelism, ratings.partitions.length / 2)
} else {
this.numUserBlocks
}
val numProductBlocks = if (this.numProductBlocks == -1) {
- math.max(sc.defaultParallelism, ratings.partitions.size / 2)
+ math.max(sc.defaultParallelism, ratings.partitions.length / 2)
} else {
this.numProductBlocks
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
index 73da899a0e..f7e3c5c501 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
@@ -350,7 +350,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
val partialWeightsArray = scaler.transform(
Vectors.dense(weightsArray.slice(start, end))).toArray
- System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.size)
+ System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.length)
i += 1
}
weights = Vectors.dense(weightsArray)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala
index 02af281fb7..a6e1767fe2 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala
@@ -74,7 +74,7 @@ private[regression] object GLMRegressionModel {
val sqlContext = SQLContext.getOrCreate(sc)
val dataRDD = sqlContext.read.parquet(datapath)
val dataArray = dataRDD.select("weights", "intercept").take(1)
- assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath")
+ assert(dataArray.length == 1, s"Unable to load $modelClass data from: $datapath")
val data = dataArray(0)
assert(data.size == 2, s"Unable to load $modelClass data from: $datapath")
data match {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
index 40440d50fc..76c32208ea 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
@@ -17,6 +17,7 @@
package org.apache.spark.mllib.tree
+import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.collection.mutable
@@ -286,6 +287,7 @@ object DecisionTree extends Serializable with Logging {
* This index is different from the index used during training a particular
* group of nodes on one call to [[findBestSplits()]].
*/
+ @tailrec
private def predictNodeIndex(
node: Node,
binnedFeatures: Array[Int],
@@ -350,7 +352,7 @@ object DecisionTree extends Serializable with Logging {
featuresForNode: Option[Array[Int]]): Unit = {
val numFeaturesPerNode = if (featuresForNode.nonEmpty) {
// Use subsampled features
- featuresForNode.get.size
+ featuresForNode.get.length
} else {
// Use all features
agg.metadata.numFeatures
@@ -411,7 +413,7 @@ object DecisionTree extends Serializable with Logging {
if (featuresForNode.nonEmpty) {
// Use subsampled features
var featureIndexIdx = 0
- while (featureIndexIdx < featuresForNode.get.size) {
+ while (featureIndexIdx < featuresForNode.get.length) {
val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx))
agg.update(featureIndexIdx, binIndex, label, instanceWeight)
featureIndexIdx += 1
@@ -483,7 +485,7 @@ object DecisionTree extends Serializable with Logging {
*/
// numNodes: Number of nodes in this group
- val numNodes = nodesForGroup.values.map(_.size).sum
+ val numNodes = nodesForGroup.values.map(_.length).sum
logDebug("numNodes = " + numNodes)
logDebug("numFeatures = " + metadata.numFeatures)
logDebug("numClasses = " + metadata.numClasses)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
index a74197278d..09017d482a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
@@ -104,9 +104,9 @@ private[tree] class VarianceAggregator()
*/
private[spark] class VarianceCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) {
- require(stats.size == 3,
+ require(stats.length == 3,
s"VarianceCalculator requires sufficient statistics array stats to be of length 3," +
- s" but was given array of length ${stats.size}.")
+ s" but was given array of length ${stats.length}.")
/**
* Make a deep copy of this [[ImpurityCalculator]].
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
index ec5d7b9189..e007ee1105 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
@@ -250,7 +250,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging {
val nodes = dataRDD.rdd.map(NodeData.apply)
// Build node data into a tree.
val trees = constructTrees(nodes)
- assert(trees.size == 1,
+ assert(trees.length == 1,
"Decision tree should contain exactly one tree but got ${trees.size} trees.")
val model = new DecisionTreeModel(trees(0), Algo.fromString(algo))
assert(model.numNodes == numNodes, s"Unable to load DecisionTreeModel data from: $datapath." +
@@ -266,7 +266,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging {
.map { case (treeId, data) =>
(treeId, constructTree(data))
}.sortBy(_._1)
- val numTrees = trees.size
+ val numTrees = trees.length
val treeIndices = trees.map(_._1).toSeq
assert(treeIndices == (0 until numTrees),
s"Tree indices must start from 0 and increment by 1, but we found $treeIndices.")
diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
index eeeabfe359..916fff14a7 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
@@ -95,7 +95,9 @@ public class JavaFPGrowthSuite implements Serializable {
try {
model.save(sc.sc(), outputPath);
- FPGrowthModel newModel = FPGrowthModel.load(sc.sc(), outputPath);
+ @SuppressWarnings("unchecked")
+ FPGrowthModel<String> newModel =
+ (FPGrowthModel<String>) FPGrowthModel.load(sc.sc(), outputPath);
List<FPGrowth.FreqItemset<String>> freqItemsets = newModel.freqItemsets().toJavaRDD()
.collect();
assertEquals(18, freqItemsets.size());
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
index 8fb8886645..a200e9454b 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -38,7 +38,7 @@ class RidgeRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]): Double = {
predictions.zip(input).map { case (prediction, expected) =>
(prediction - expected.label) * (prediction - expected.label)
- }.reduceLeft(_ + _) / predictions.size
+ }.sum / predictions.size
}
test("ridge regression can help avoid overfitting") {
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala
index 50441816ec..0921fdba33 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala
@@ -164,7 +164,7 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase {
// number of batches seen so far does not exceed testWindow, expect counts to continue growing
for (i <- 0 until testWindow) {
- assert(outputCounts.drop(2 * i).take(2).forall(_ == (i + 1) * pointsPerBatch / 2))
+ assert(outputCounts.slice(2 * i, 2 * i + 2).forall(_ == (i + 1) * pointsPerBatch / 2))
}
// number of batches seen exceeds testWindow, expect counts to be constant