From 7836e2fe7b2d55008eb63ec56d207fbe28c0c2ab Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 1 Apr 2014 22:31:27 -0700 Subject: [PATCH 01/12] initial refactoring of matrices backed by RDDs --- .../apache/spark/mllib/linalg/Matrices.scala | 65 +++++ .../spark/mllib/linalg/MatrixEntry.scala | 10 +- .../org/apache/spark/mllib/linalg/SVD.scala | 4 +- .../linalg/SingularValueDecomposition.scala | 4 + .../linalg/rdd/CoordinateRDDMatrix.scala | 73 ++++++ .../linalg/rdd/IndexedRowRDDMatrix.scala | 56 +++++ .../spark/mllib/linalg/rdd/RDDMatrix.scala | 30 +++ .../mllib/linalg/rdd/RDDMatrixEntry.scala | 26 ++ .../spark/mllib/linalg/rdd/RDDMatrixRow.scala | 23 ++ .../spark/mllib/linalg/rdd/RowRDDMatrix.scala | 232 ++++++++++++++++++ .../org/apache/spark/mllib/util/LAUtils.scala | 4 +- .../apache/spark/mllib/linalg/PCASuite.scala | 2 +- .../apache/spark/mllib/linalg/SVDSuite.scala | 2 +- .../linalg/rdd/CoordinateRDDMatrixSuite.scala | 69 ++++++ .../mllib/linalg/rdd/RowRDDMatrixSuite.scala | 90 +++++++ 15 files changed, 679 insertions(+), 11 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixEntry.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixRow.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala 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 new file mode 100644 index 0000000000000..6c6482bb2c6d8 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} + +/** + * Trait for matrix. + */ +trait Matrix extends Serializable { + + /** Number of rows. */ + def m: Int + + /** Number of columns. */ + def n: Int + + /** Converts to a dense array in column major. */ + def toArray: Array[Double] + + /** Converts to a breeze matrix. */ + private[mllib] def toBreeze: BM[Double] +} + +/** + * Column majored dense matrix. + * + * @param m + * @param n + * @param values + */ +class DenseMatrix(val m: Int, val n: Int, val values: Array[Double]) extends Matrix { + + require(values.length == m * n) + + def toArray: Array[Double] = values + + private[mllib] def toBreeze: BM[Double] = new BDM[Double](m, n, values) +} + +object Matrices { + + def dense(m: Int, n: Int, values: Array[Double]): Matrix = { + new DenseMatrix(m, n, values) + } + + private[mllib] def fromBreeze(breeze: BDM[Double]): Matrix = { + null + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala index 416996fcbe760..e8e013d65d7c2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala @@ -18,10 +18,10 @@ package org.apache.spark.mllib.linalg /** - * Class that represents an entry in a sparse matrix of doubles. + * Class that represents an entry in a matrix of doubles. * - * @param i row index (0 indexing used) - * @param j column index (0 indexing used) - * @param mval value of entry in matrix + * @param i row index (0-based indexing) + * @param j column index (0-based indexing) + * @param value value of the entry */ -case class MatrixEntry(val i: Int, val j: Int, val mval: Double) +case class MatrixEntry(val i: Int, val j: Int, val value: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index 3e7cc648d1d37..ac665996fe26d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -285,7 +285,7 @@ class SVD { // Compute A^T A, assuming rows are sparse enough to fit in memory val rows = data.map(entry => - (entry.i, (entry.j, entry.mval))).groupByKey() + (entry.i, (entry.j, entry.value))).groupByKey() val emits = rows.flatMap { case (rowind, cols) => cols.flatMap { @@ -339,7 +339,7 @@ class SVD { if (computeU) { // Multiply A by VS^-1 - val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) + val aCols = data.map(entry => (entry.j, (entry.i, entry.value))) val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) val retUdata = aCols.join(bRows).map { case (key, ((rowInd, rowVal), (colInd, colVal))) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala new file mode 100644 index 0000000000000..7c2c949714348 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -0,0 +1,4 @@ +package org.apache.spark.mllib.linalg + +/** Represents SVD factors */ +case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala new file mode 100644 index 0000000000000..d8a72aed37dda --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vectors + +/** + * Represents a matrix in coordinate list format. + * + * @param entries matrix entries + * @param m number of rows (default: -1L, which means unknown) + * @param n number of column (default: -1L, which means unknown) + */ +class CoordinateRDDMatrix( + val entries: RDD[RDDMatrixEntry], + m: Long = -1L, + n: Long = -1L) extends RDDMatrix { + + private var _m = m + private var _n = n + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (_n < 0) { + computeSize() + } + _n + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (_m < 0) { + computeSize() + } + _m + } + + private def computeSize() { + val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => + (math.max(i1, i2), math.max(j1, j2)) + } + // There may be empty columns at the very right and empty rows at the very bottom. + _m = math.max(_m, m1 + 1L) + _n = math.max(_n, n1 + 1L) + } + + def toIndexedRowRDDMatrix(): IndexedRowRDDMatrix = { + val n = numCols().toInt + val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) + .groupByKey() + .map { case (i, vectorEntries) => + RDDMatrixRow(i, Vectors.sparse(n, vectorEntries)) + } + new IndexedRowRDDMatrix(indexedRows, numRows(), numCols()) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala new file mode 100644 index 0000000000000..eab846a1604c9 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +import org.apache.spark.rdd.RDD + +/** + * Represents a row-oriented RDDMatrix with indexed rows. + * + * @param rows + * @param m + * @param n + */ +class IndexedRowRDDMatrix( + val rows: RDD[RDDMatrixRow], + m: Long = -1L, + n: Long = -1L) extends RDDMatrix { + + private var _m = m + private var _n = n + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (_n < 0) { + _n = rows.first().vector.size + } + _n + } + + override def numRows(): Long = { + if (_m < 0) { + _m = rows.map(_.index).reduce(math.max) + 1 + } + _m + } + + /** Drops row indices and converts this to a RowRDDMatrix. */ + def compressRows(): RowRDDMatrix = { + new RowRDDMatrix(rows.map(_.vector), -1, _n) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrix.scala new file mode 100644 index 0000000000000..9452593ed091f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrix.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +/** + * Represents a matrix backed by one or more RDDs. + */ +trait RDDMatrix extends Serializable { + + /** Gets or computes the number of rows. */ + def numRows(): Long + + /** Gets or computes the number of columns. */ + def numCols(): Long +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixEntry.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixEntry.scala new file mode 100644 index 0000000000000..79dadf1fbb1fe --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixEntry.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +/** + * Represents an entry in an RDDMatrix. + * @param i row index + * @param j column index + * @param value value of the entry + */ +case class RDDMatrixEntry(i: Long, j: Long, value: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixRow.scala new file mode 100644 index 0000000000000..b964fc206f9d3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixRow.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +import org.apache.spark.mllib.linalg.Vector + +/** Represents a row of RowRDDMatrix. */ +case class RDDMatrixRow(index: Long, vector: Vector) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala new file mode 100644 index 0000000000000..aa9a58fef42ad --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +import java.util + +import scala.util.control.Breaks._ + +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.numerics.{sqrt => brzSqrt} +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.mllib.linalg._ +import org.apache.spark.rdd.RDD + +/** + * Represents a row-oriented RDDMatrix with no meaningful row indices. + * + * @param rows rows stored as an RDD[Vector] + * @param m number of rows + * @param n number of columns + */ +class RowRDDMatrix( + val rows: RDD[Vector], + m: Long = -1L, + n: Long = -1) extends RDDMatrix { + + private var _m = m + private var _n = n + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (_n < 0) { + _n = rows.first().size + } + _n + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (_m < 0) { + _m = rows.count() + } + _m + } + + /** + * Computes the gram matrix `A^T A`. + */ + def gram(): Matrix = { + val n = numCols().toInt + val nt: Int = n * (n + 1) / 2 + + // Compute the upper triangular part of the gram matrix. + val GU = rows.aggregate(new BDV[Double](new Array[Double](nt)))( + seqOp = (U, v) => { + RowRDDMatrix.dspr(1.0, v, U.data) + U + }, + combOp = (U1, U2) => U1 += U2 + ) + + RowRDDMatrix.triuToFull(n, GU.data) + } + + + /** + * Singular Value Decomposition for Tall and Skinny matrices. + * Given an m x n matrix A, this will compute matrices U, S, V such that + * A = U * S * V' + * + * There is no restriction on m, but we require n^2 doubles to fit in memory. + * Further, n should be less than m. + * + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), + * from which we recover S and V. + * Then we compute U via easy matrix multiplication + * as U = A * V * S^-1 + * + * Only the k largest singular values and associated vectors are found. + * If there are k such values, then the dimensions of the return will be: + * + * S is k x k and diagonal, holding the singular values on diagonal + * U is m x k and satisfies U'U = eye(k) + * V is n x k and satisfies V'V = eye(k) + * + * The return values are as lean as possible: an RDD of rows for U, + * a simple array for sigma, and a dense 2d matrix array for V + * + * @param matrix dense matrix to factorize + * @return Three matrices: U, S, V such that A = USV^T + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[RowRDDMatrix, Matrix] = { + + val n = numCols().toInt + + require(k >= 0 && k <= n, s"Request up to n singular values k=$k n=$n.") + + val G = gram() + + // TODO: Use sparse SVD instead. + val (u: BDM[Double], sigmaSquares: BDV[Double], v: BDM[Double]) = + brzSvd(G.toBreeze.asInstanceOf[BDM[Double]]) + val sigmas: BDV[Double] = brzSqrt(sigmaSquares) + + // Determine effective rank. + val sigma0 = sigmas(0) + val threshold = rCond * sigma0 + var i = 0 + breakable { + while (i < k) { + if (sigmas(i) < threshold) { + break() + } + i += 1 + } + } + val sk = i + + val s = Vectors.dense(util.Arrays.copyOfRange(sigmas.data, 0, sk)) + val V = Matrices.dense(n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + + if (computeU) { + val N = new BDM[Double](sk, n, util.Arrays.copyOfRange(v.data, 0, sk * n)) + var i = 0 + var j = 0 + while (i < sk) { + j = 0 + val sigma = sigmas(i) + while (j < n) { + N(i, j) /= sigma + j += 1 + } + i += 1 + } + val Nb = rows.context.broadcast(N) + val rowsU = rows.map { row => + Vectors.fromBreeze(Nb.value * row.toBreeze) + } + SingularValueDecomposition(new RowRDDMatrix(rowsU, _m, n), s, V) + } else { + SingularValueDecomposition(null, s, V) + } + } +} + +object RowRDDMatrix { + + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. + * + * @param U the upper triangular part of the matrix packed in an array (column major) + */ + private def dspr(alpha: Double, v: Vector, U: Array[Double]): Unit = { + // TODO: Find a better home (breeze?) for this method. + val n = v.size + v match { + case dv: DenseVector => + blas.dspr("U", n, 1.0, dv.values, 1, U) + case sv: SparseVector => + val indices = sv.indices + val values = sv.values + val nnz = indices.length + var colStartIdx = 0 + var prevCol = 0 + var col = 0 + var j = 0 + var i = 0 + var av = 0.0 + while (j < nnz) { + col = indices(j) + // Skip empty columns. + colStartIdx += (col - prevCol) * (col + prevCol + 1) / 2 + col = indices(j) + av = alpha * values(j) + i = 0 + while (i <= j) { + U(colStartIdx + indices(i)) += av * values(i) + i += 1 + } + j += 1 + prevCol = col + } + } + } + + /** + * Fills a full square matrix from its upper triangular part. + */ + private def triuToFull(n: Int, U: Array[Double]): Matrix = { + val G = new BDM[Double](n, n) + + var row = 0 + var col = 0 + var idx = 0 + var value = 0.0 + while (col < n) { + row = 0 + while (row < col) { + value = U(idx) + G(row, col) = value + G(col, row) = value + idx += 1 + row += 1 + } + G(col, col) = U(idx) + idx += 1 + col +=1 + } + + Matrices.dense(n, n, G.data) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala index afe081295bfae..3d64ad68c77d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala @@ -34,7 +34,7 @@ object LAUtils { def sparseToTallSkinnyDense(sp: SparseMatrix): TallSkinnyDenseMatrix = { val m = sp.m val n = sp.n - val rows = sp.data.map(x => (x.i, (x.j, x.mval))).groupByKey().map { + val rows = sp.data.map(x => (x.i, (x.j, x.value))).groupByKey().map { case (i, cols) => val rowArray = Array.ofDim[Double](n) var j = 0 @@ -58,7 +58,7 @@ object LAUtils { val n = a.n val data = a.rows.flatMap { mrow => Array.tabulate(n)(j => MatrixEntry(mrow.i, j, mrow.data(j))) - .filter(x => x.mval != 0) + .filter(x => x.value != 0) } SparseMatrix(data, m, n) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala index 5e5086b1bf73e..39ea3389a58d7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala @@ -48,7 +48,7 @@ class PCASuite extends FunSuite with BeforeAndAfterAll { def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { val data = matrix.data val ret = DoubleMatrix.zeros(matrix.m, matrix.n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) + matrix.data.collect().map(x => ret.put(x.i, x.j, x.value)) ret } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala index 20e2b0f84be06..f9026437529bb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala @@ -52,7 +52,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val m = matrix.m val n = matrix.n val ret = DoubleMatrix.zeros(m, n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) + matrix.data.collect().map(x => ret.put(x.i, x.j, x.value)) ret } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala new file mode 100644 index 0000000000000..7911c4103e434 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors + +class CoordinateRDDMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 4 + var mat: CoordinateRDDMatrix = _ + + override def beforeAll() { + super.beforeAll() + val entries = sc.parallelize(Seq( + (0, 0, 1.0), + (0, 1, 2.0), + (1, 1, 3.0), + (1, 2, 4.0), + (2, 2, 5.0), + (2, 3, 6.0), + (3, 0, 7.0), + (3, 3, 8.0), + (4, 1, 9.0)), 3).map { case (i, j, value) => + RDDMatrixEntry(i, j, value) + } + mat = new CoordinateRDDMatrix(entries) + } + + test("size") { + assert(mat.numRows() === 4) + assert(mat.numCols() === 5) + } + + test("toIndexedRowRDDMatrix") { + val indexedRows = mat + .toIndexedRowRDDMatrix() + .rows + .map(row => (row.index, row.vector)) + .collect() + .sortBy(_._1) + .toSeq + assert(indexedRows === Seq( + (0, Vectors.dense(1.0, 2.0, 0.0, 0.0)), + (1, Vectors.dense(0.0, 3.0, 4.0, 0.0)), + (2, Vectors.dense(0.0, 0.0, 5.0, 6.0)), + (3, Vectors.dense(7.0, 0.0, 0.0, 8.0)), + (4, Vectors.dense(0.0, 9.0, 0.0, 0.0)) + )) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala new file mode 100644 index 0000000000000..5f74203a364ea --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.rdd + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, diag => brzDiag} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.{Matrices, Vectors} + +class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val arr = Array(0.0, 3.0, 6.0, 9.0, 1.0, 4.0, 7.0, 0.0, 2.0, 5.0, 8.0, 1.0) + val denseData = Seq( + Vectors.dense(0.0, 1.0, 2.0), + Vectors.dense(3.0, 4.0, 5.0), + Vectors.dense(6.0, 7.0, 8.0), + Vectors.dense(9.0, 0.0, 1.0) + ) + val sparseData = Seq( + Vectors.sparse(3, Seq((1, 1.0), (2, 2.0))), + Vectors.sparse(3, Seq((0, 3.0), (1, 4.0), (2, 5.0))), + Vectors.sparse(3, Seq((0, 6.0), (1, 7.0), (2, 8.0))), + Vectors.sparse(3, Seq((0, 9.0), (2, 1.0))) + ) + + var denseMat: RowRDDMatrix = _ + var sparseMat: RowRDDMatrix = _ + + override def beforeAll() { + super.beforeAll() + denseMat = new RowRDDMatrix(sc.parallelize(denseData, 2)) + sparseMat = new RowRDDMatrix(sc.parallelize(sparseData, 2)) + } + + test("size") { + assert(denseMat.numRows() === m) + assert(denseMat.numCols() === n) + assert(sparseMat.numRows() === m) + assert(sparseMat.numCols() === n) + } + + test("gram") { + val expected = + Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.gram() + assert(G.toBreeze === expected.toBreeze) + } + } + + test("svd") { + val A = new BDM[Double](m, n, arr) + for (mat <- Seq(denseMat, sparseMat)) { + val svd = mat.computeSVD(n, computeU = true) + val U = svd.U + val brzSigma = svd.s.toBreeze.asInstanceOf[BDV[Double]] + val brzV = svd.V.toBreeze.asInstanceOf[BDM[Double]] + val rows = U.rows.collect() + val brzUt = new BDM[Double](n, m, rows.flatMap(r => r.toArray)) + assert(closeToZero(brzUt.t * brzDiag(brzSigma) * brzV.t - A)) + val VtV: BDM[Double] = brzV.t * brzV + assert(closeToZero(VtV - BDM.eye[Double](n))) + val UtU = U.gram().toBreeze.asInstanceOf[BDM[Double]] + assert(closeToZero(UtU - BDM.eye[Double](n))) + } + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } +} From 4cf679c9af585ed5342bcd925935cb096a53ad08 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 13:20:51 -0700 Subject: [PATCH 02/12] port pca to RowRDDMatrix, and add multiply and covariance --- .../apache/spark/mllib/linalg/Matrices.scala | 11 +- .../linalg/SingularValueDecomposition.scala | 19 ++- .../linalg/rdd/IndexedRowRDDMatrix.scala | 6 +- .../spark/mllib/linalg/rdd/RowRDDMatrix.scala | 151 ++++++++++++++---- .../mllib/linalg/rdd/RowRDDMatrixSuite.scala | 52 +++++- 5 files changed, 201 insertions(+), 38 deletions(-) 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 6c6482bb2c6d8..5ecf7c163a823 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 @@ -59,7 +59,14 @@ object Matrices { new DenseMatrix(m, n, values) } - private[mllib] def fromBreeze(breeze: BDM[Double]): Matrix = { - null + private[mllib] def fromBreeze(breeze: BM[Double]): Matrix = { + breeze match { + case dm: BDM[Double] => + require(dm.majorStride == dm.rows) + new DenseMatrix(dm.rows, dm.cols, dm.data) + case _ => + throw new UnsupportedOperationException( + s"Do not support conversion from type ${breeze.getClass.getName}.") + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 7c2c949714348..46b105457430c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -1,4 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.mllib.linalg -/** Represents SVD factors */ +/** Represents singular value decomposition (SVD) factors. */ case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala index eab846a1604c9..c0ead63a84c7e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala @@ -22,9 +22,9 @@ import org.apache.spark.rdd.RDD /** * Represents a row-oriented RDDMatrix with indexed rows. * - * @param rows - * @param m - * @param n + * @param rows indexed rows of this matrix + * @param m number of rows, where a negative number means unknown + * @param n number of cols, where a negative number means unknown */ class IndexedRowRDDMatrix( val rows: RDD[RDDMatrixRow], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala index aa9a58fef42ad..3218033047e7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala @@ -27,6 +27,7 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD +import org.apache.spark.Logging /** * Represents a row-oriented RDDMatrix with no meaningful row indices. @@ -38,7 +39,7 @@ import org.apache.spark.rdd.RDD class RowRDDMatrix( val rows: RDD[Vector], m: Long = -1L, - n: Long = -1) extends RDDMatrix { + n: Long = -1) extends RDDMatrix with Logging { private var _m = m private var _n = n @@ -60,9 +61,9 @@ class RowRDDMatrix( } /** - * Computes the gram matrix `A^T A`. + * Computes the Gramian matrix `A^T A`. */ - def gram(): Matrix = { + def computeGramianMatrix(): Matrix = { val n = numCols().toInt val nt: Int = n * (n + 1) / 2 @@ -80,31 +81,30 @@ class RowRDDMatrix( /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. * - * There is no restriction on m, but we require n^2 doubles to fit in memory. + * There is no restriction on m, but we require `n^2` doubles to fit in memory. * Further, n should be less than m. - * + * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. * - * Only the k largest singular values and associated vectors are found. + * At most k largest non-zero singular values and associated vectors are returned. * If there are k such values, then the dimensions of the return will be: * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * The return values are as lean as possible: an RDD of rows for U, - * a simple array for sigma, and a dense 2d matrix array for V + * U is a RowRDDMatrix of size m x k that satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a Matrix of size n x k that satisfies V'V = eye(k). * - * @param matrix dense matrix to factorize - * @return Three matrices: U, S, V such that A = USV^T + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) */ def computeSVD( k: Int, @@ -113,9 +113,9 @@ class RowRDDMatrix( val n = numCols().toInt - require(k >= 0 && k <= n, s"Request up to n singular values k=$k n=$n.") + require(k > 0 && k <= n, s"Request up to n singular values k=$k n=$n.") - val G = gram() + val G = computeGramianMatrix() // TODO: Use sparse SVD instead. val (u: BDM[Double], sigmaSquares: BDV[Double], v: BDM[Double]) = @@ -152,15 +152,110 @@ class RowRDDMatrix( } i += 1 } - val Nb = rows.context.broadcast(N) - val rowsU = rows.map { row => - Vectors.fromBreeze(Nb.value * row.toBreeze) - } - SingularValueDecomposition(new RowRDDMatrix(rowsU, _m, n), s, V) + val U = this.multiply(Matrices.fromBreeze(N)) + SingularValueDecomposition(U, s, V) } else { SingularValueDecomposition(null, s, V) } } + + /** + * Computes the covariance matrix, treating each row as an observation. + * @return a local dense matrix of size n x n + */ + def computeCovariance(): Matrix = { + val n = numCols().toInt + + if (n > 10000) { + val mem = n * n * java.lang.Double.SIZE / java.lang.Byte.SIZE + logWarning(s"The number of columns $n is greater than 10000! " + + s"We need at least $mem bytes of memory.") + } + + val (m, mean) = rows.aggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( + seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), + combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) + ) + + // Update _m if it is not set, or verify its value. + if (_m < 0L) { + _m = m + } else { + require(_m == m, + s"The number of rows $m is different from what specified or previously computed: ${_m}.") + } + + mean :/= m.toDouble + + // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is + // large but Cov(X, Y) is small, but it is good for sparse computation. + // TODO: find a fast and stable way for sparse data. + + val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + + var i = 0 + var j = 0 + val m1 = m - 1.0 + var alpha = 0.0 + while (i < n) { + alpha = m / m1 * mean(i) + j = 0 + while (j < n) { + G(i, j) = G(i, j) / m1 - alpha * mean(j) + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(G) + } + + /** + * Computes the top k principal components. + * Rows correspond to observations and columns correspond to variables. + * The principal components are stored a local matrix of size n-by-k. + * Each column corresponds for one principal component, + * and the columns are in descending order of component variance. + * + * @param k number of top principal components. + * @return a matrix of size n-by-k, whose columns are principal components + */ + def computePrincipalComponents(k: Int): Matrix = { + val n = numCols().toInt + require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") + + val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] + + val (u: BDM[Double], _, _) = brzSvd(Cov) + + if (k == n) { + Matrices.dense(n, k, u.data) + } else { + Matrices.dense(n, k, util.Arrays.copyOfRange(u.data, 0, n * k)) + } + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return a RowRDDMatrix representing the product, which preserves partitioning + */ + def multiply(B: Matrix): RowRDDMatrix = { + val n = numCols().toInt + require(n == B.m, s"Dimension mismatch: $n vs ${B.m}") + + require(B.isInstanceOf[DenseMatrix], + s"Only support dense matrix at this time but found ${B.getClass.getName}.") + + val Bb = rows.context.broadcast(B) + val AB = rows.mapPartitions({ iter => + val Bi = Bb.value.toBreeze.asInstanceOf[BDM[Double]] + iter.map(v => Vectors.fromBreeze(Bi.t * v.toBreeze)) + }, preservesPartitioning = true) + + new RowRDDMatrix(AB, _m, B.n) + } } object RowRDDMatrix { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala index 5f74203a364ea..2b85481df4d6d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.linalg.rdd import org.scalatest.FunSuite -import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, diag => brzDiag} +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, diag => brzDiag, norm => brzNorm} import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.linalg.{Matrices, Vectors} +import org.apache.spark.mllib.linalg.{Matrices, Vectors, Matrix} class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { @@ -42,6 +42,10 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { Vectors.sparse(3, Seq((0, 9.0), (2, 1.0))) ) + val principalComponents = Matrices.dense(n, n, + Array(0.0, math.sqrt(2.0) / 2.0, math.sqrt(2.0) / 2.0, 1.0, 0.0, 0.0, + 0.0, math.sqrt(2.0) / 2.0, - math.sqrt(2.0) / 2.0)) + var denseMat: RowRDDMatrix = _ var sparseMat: RowRDDMatrix = _ @@ -62,7 +66,7 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { val expected = Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) for (mat <- Seq(denseMat, sparseMat)) { - val G = mat.gram() + val G = mat.computeGramianMatrix() assert(G.toBreeze === expected.toBreeze) } } @@ -79,7 +83,7 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { assert(closeToZero(brzUt.t * brzDiag(brzSigma) * brzV.t - A)) val VtV: BDM[Double] = brzV.t * brzV assert(closeToZero(VtV - BDM.eye[Double](n))) - val UtU = U.gram().toBreeze.asInstanceOf[BDM[Double]] + val UtU = U.computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] assert(closeToZero(UtU - BDM.eye[Double](n))) } } @@ -87,4 +91,44 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { def closeToZero(G: BDM[Double]): Boolean = { G.valuesIterator.map(math.abs).sum < 1e-6 } + + def closeToZero(v: BDV[Double]): Boolean = { + brzNorm(v, 1.0) < 1e-6 + } + + def assertPrincipalComponentsEqual(a: Matrix, b: Matrix, k: Int) { + val brzA = a.toBreeze.asInstanceOf[BDM[Double]] + val brzB = b.toBreeze.asInstanceOf[BDM[Double]] + assert(brzA.rows === brzB.rows) + for (j <- 0 until k) { + val aj = brzA(::, j) + val bj = brzB(::, j) + assert(closeToZero(aj - bj) || closeToZero(aj + bj), + s"The $j-th components mismatch: $aj and $bj") + } + } + + test("pca") { + for (mat <- Seq(denseMat, sparseMat); k <- 1 to n) { + val pc = denseMat.computePrincipalComponents(k) + assert(pc.m === n) + assert(pc.n === k) + assertPrincipalComponentsEqual(pc, principalComponents, k) + } + } + + test("multiply a local matrix") { + val B = Matrices.dense(n, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val AB = mat.multiply(B) + assert(AB.numRows() === m) + assert(AB.numCols() === 2) + assert(AB.rows.collect().toSeq === Seq( + Vectors.dense(5.0, 14.0), + Vectors.dense(14.0, 50.0), + Vectors.dense(23.0, 86.0), + Vectors.dense(2.0, 32.0) + )) + } + } } From b8b6ac348f8620b2cd5b52545418fa63b1fe7ae4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 13:33:32 -0700 Subject: [PATCH 03/12] Remove old code --- .../spark/examples/mllib/SparkPCA.scala | 3 - .../spark/examples/mllib/SparkSVD.scala | 2 - .../spark/mllib/linalg/MatrixEntry.scala | 27 -- .../apache/spark/mllib/linalg/MatrixRow.scala | 26 -- .../apache/spark/mllib/linalg/MatrixSVD.scala | 29 -- .../org/apache/spark/mllib/linalg/PCA.scala | 120 ------ .../org/apache/spark/mllib/linalg/SVD.scala | 397 ------------------ .../spark/mllib/linalg/SparseMatrix.scala | 30 -- .../mllib/linalg/TallSkinnyDenseMatrix.scala | 30 -- .../mllib/linalg/TallSkinnyMatrixSVD.scala | 31 -- .../spark/mllib/linalg/rdd/RowRDDMatrix.scala | 1 - .../org/apache/spark/mllib/util/LAUtils.scala | 65 --- .../apache/spark/mllib/linalg/PCASuite.scala | 124 ------ .../apache/spark/mllib/linalg/SVDSuite.scala | 194 --------- 14 files changed, 1079 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala index d4e08c5e12d81..85fc5b3a08e12 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala @@ -18,9 +18,6 @@ package org.apache.spark.examples.mllib import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.PCA -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix import org.apache.spark.mllib.util._ diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala index 2933cec497b37..8068c0da96920 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala @@ -19,8 +19,6 @@ package org.apache.spark.examples.mllib import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.SVD -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix /** * Compute SVD of an example matrix diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala deleted file mode 100644 index e8e013d65d7c2..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -/** - * Class that represents an entry in a matrix of doubles. - * - * @param i row index (0-based indexing) - * @param j column index (0-based indexing) - * @param value value of the entry - */ -case class MatrixEntry(val i: Int, val j: Int, val value: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala deleted file mode 100644 index 2608a67bfe260..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -/** - * Class that represents a row of a dense matrix - * - * @param i row index (0 indexing used) - * @param data entries of the row - */ -case class MatrixRow(val i: Int, val data: Array[Double]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala deleted file mode 100644 index 319f82b449096..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -/** - * Class that represents the SV decomposition of a matrix - * - * @param U such that A = USV^T - * @param S such that A = USV^T - * @param V such that A = USV^T - */ -case class MatrixSVD(val U: SparseMatrix, - val S: SparseMatrix, - val V: SparseMatrix) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala deleted file mode 100644 index fe5b3f6c7e463..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import org.apache.spark.rdd.RDD - - -import org.jblas.DoubleMatrix - - -/** - * Class used to obtain principal components - */ -class PCA { - private var k = 1 - - /** - * Set the number of top-k principle components to return - */ - def setK(k: Int): PCA = { - this.k = k - this - } - - /** - * Compute PCA using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Compute PCA using the parameters currently set - * See computePCA() for more details - */ - def compute(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform PCA on - * @return An nxk matrix with principal components in columns. Columns are inner arrays - */ - private def computePCA(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - val m = matrix.m - val n = matrix.n - - if (m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a well-formed matrix: m=$m n=$n") - } - - computePCA(matrix.rows.map(_.data)) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform pca on - * @return An nxk matrix of principal components - */ - private def computePCA(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - val n = matrix.first.size - - // compute column sums and normalize matrix - val colSumsTemp = matrix.map((_, 1)).fold((Array.ofDim[Double](n), 0)) { - (a, b) => - val am = new DoubleMatrix(a._1) - val bm = new DoubleMatrix(b._1) - am.addi(bm) - (a._1, a._2 + b._2) - } - - val m = colSumsTemp._2 - val colSums = colSumsTemp._1.map(x => x / m) - - val data = matrix.map { - x => - val row = Array.ofDim[Double](n) - var i = 0 - while (i < n) { - row(i) = x(i) - colSums(i) - i += 1 - } - row - } - - val (u, s, v) = new SVD().setK(k).compute(data) - v - } -} - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala deleted file mode 100644 index ac665996fe26d..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ /dev/null @@ -1,397 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -/** - * Class used to obtain singular value decompositions - */ -class SVD { - private var k = 1 - private var computeU = true - - // All singular values smaller than rCond * sigma(0) - // are treated as zero, where sigma(0) is the largest singular value. - private var rCond = 1e-9 - - /** - * Set the number of top-k singular vectors to return - */ - def setK(k: Int): SVD = { - this.k = k - this - } - - /** - * Sets the reciprocal condition number (rCond). All singular values - * smaller than rCond * sigma(0) are treated as zero, - * where sigma(0) is the largest singular value. - */ - def setReciprocalConditionNumber(smallS: Double): SVD = { - this.rCond = smallS - this - } - - /** - * Should U be computed? - */ - def setComputeU(compU: Boolean): SVD = { - this.computeU = compU - this - } - - /** - * Compute SVD using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - denseSVD(matrix) - } - - /** - * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T - * U is a row-by-row dense matrix - * S is a simple double array of singular values - * V is a 2d array matrix - * See [[denseSVD]] for more documentation - */ - def compute(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - denseSVD(matrix) - } - - /** - * See full paramter definition of sparseSVD for more description. - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - def compute(matrix: SparseMatrix): MatrixSVD = { - sparseSVD(matrix) - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * @param matrix dense matrix to factorize - * @return See [[TallSkinnyMatrixSVD]] for the output matrices and arrays - */ - private def denseSVD(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Request up to n singular values n=$n k=$k") - } - - val rowIndices = matrix.rows.map(_.i) - - // compute SVD - val (u, sigma, v) = denseSVD(matrix.rows.map(_.data)) - - if (computeU) { - // prep u for returning - val retU = TallSkinnyDenseMatrix( - u.zip(rowIndices).map { - case (row, i) => MatrixRow(i, row) - }, - m, - k) - - TallSkinnyMatrixSVD(retU, sigma, v) - } else { - TallSkinnyMatrixSVD(null, sigma, v) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * The return values are as lean as possible: an RDD of rows for U, - * a simple array for sigma, and a dense 2d matrix array for V - * - * @param matrix dense matrix to factorize - * @return Three matrices: U, S, V such that A = USV^T - */ - private def denseSVD(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - val n = matrix.first.size - - if (k < 1 || k > n) { - throw new IllegalArgumentException( - "Request up to n singular values k=$k n=$n") - } - - // Compute A^T A - val fullata = matrix.mapPartitions { - iter => - val localATA = Array.ofDim[Double](n, n) - while (iter.hasNext) { - val row = iter.next() - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - localATA(i)(j) += row(i) * row(j) - j += 1 - } - i += 1 - } - } - Iterator(localATA) - }.fold(Array.ofDim[Double](n, n)) { - (a, b) => - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - a(i)(j) += b(i)(j) - j += 1 - } - i += 1 - } - a - } - - // Construct jblas A^T A locally - val ata = new DoubleMatrix(fullata) - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x / svd(1).get(0) > rCond) - - val sk = Math.min(k, sigmas.size) - val sigma = sigmas.take(sk) - - // prepare V for returning - val retV = Array.tabulate(n, sk)((i, j) => V.get(i, j)) - - if (computeU) { - // Compute U as U = A V S^-1 - // Compute VS^-1 - val vsinv = new DoubleMatrix(Array.tabulate(n, sk)((i, j) => V.get(i, j) / sigma(j))) - val retU = matrix.map { - x => - val v = new DoubleMatrix(Array(x)) - v.mmul(vsinv).data - } - (retU, sigma, retV) - } else { - (null, sigma, retV) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny sparse matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require O(n^2) doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * All input and output is expected in sparse matrix format, 0-indexed - * as tuples of the form ((i,j),value) all in RDDs using the - * SparseMatrix class - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - private def sparseSVD(matrix: SparseMatrix): MatrixSVD = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Must request up to n singular values") - } - - // Compute A^T A, assuming rows are sparse enough to fit in memory - val rows = data.map(entry => - (entry.i, (entry.j, entry.value))).groupByKey() - val emits = rows.flatMap { - case (rowind, cols) => - cols.flatMap { - case (colind1, mval1) => - cols.map { - case (colind2, mval2) => - ((colind1, colind2), mval1 * mval2) - } - } - }.reduceByKey(_ + _) - - // Construct jblas A^T A locally - val ata = DoubleMatrix.zeros(n, n) - for (entry <- emits.collect()) { - ata.put(entry._1._1, entry._1._2, entry._2) - } - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - // This will be updated to rcond - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9) - - if (sigmas.size < k) { - throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size) - } - - val sigma = sigmas.take(k) - - val sc = data.sparkContext - - // prepare V for returning - val retVdata = sc.makeRDD( - Array.tabulate(V.rows, sigma.length) { - (i, j) => - MatrixEntry(i, j, V.get(i, j)) - }.flatten) - val retV = SparseMatrix(retVdata, V.rows, sigma.length) - - val retSdata = sc.makeRDD(Array.tabulate(sigma.length) { - x => MatrixEntry(x, x, sigma(x)) - }) - - val retS = SparseMatrix(retSdata, sigma.length, sigma.length) - - // Compute U as U = A V S^-1 - // turn V S^-1 into an RDD as a sparse matrix - val vsirdd = sc.makeRDD(Array.tabulate(V.rows, sigma.length) { - (i, j) => ((i, j), V.get(i, j) / sigma(j)) - }.flatten) - - if (computeU) { - // Multiply A by VS^-1 - val aCols = data.map(entry => (entry.j, (entry.i, entry.value))) - val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) - val retUdata = aCols.join(bRows).map { - case (key, ((rowInd, rowVal), (colInd, colVal))) => - ((rowInd, colInd), rowVal * colVal) - }.reduceByKey(_ + _).map { - case ((row, col), mval) => MatrixEntry(row, col, mval) - } - - val retU = SparseMatrix(retUdata, m, sigma.length) - MatrixSVD(retU, retS, retV) - } else { - MatrixSVD(null, retS, retV) - } - } -} - -/** - * Top-level methods for calling sparse Singular Value Decomposition - * NOTE: All matrices are 0-indexed - */ -object SVD { - def main(args: Array[String]) { - if (args.length < 8) { - println("Usage: SVD " + - " ") - System.exit(1) - } - - val (master, inputFile, m, n, k, output_u, output_s, output_v) = - (args(0), args(1), args(2).toInt, args(3).toInt, - args(4).toInt, args(5), args(6), args(7)) - - val sc = new SparkContext(master, "SVD") - - val rawData = sc.textFile(inputFile) - val data = rawData.map { - line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) - } - - val decomposed = new SVD().setK(k).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("Computed " + s.collect().length + " singular values and vectors") - u.saveAsTextFile(output_u) - s.saveAsTextFile(output_s) - v.saveAsTextFile(output_v) - System.exit(0) - } -} - - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala deleted file mode 100644 index cbd1a2a5a4bd8..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import org.apache.spark.rdd.RDD - - -/** - * Class that represents a sparse matrix - * - * @param data RDD of nonzero entries - * @param m number of rows - * @param n numner of columns - */ -case class SparseMatrix(val data: RDD[MatrixEntry], val m: Int, val n: Int) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala deleted file mode 100644 index e4ef3c58e8680..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import org.apache.spark.rdd.RDD - - -/** - * Class that represents a dense matrix - * - * @param rows RDD of rows - * @param m number of rows - * @param n number of columns - */ -case class TallSkinnyDenseMatrix(val rows: RDD[MatrixRow], val m: Int, val n: Int) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala deleted file mode 100644 index b3a450e92394e..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -/** - * Class that represents the singular value decomposition of a matrix - * - * @param U such that A = USV^T is a TallSkinnyDenseMatrix - * @param S such that A = USV^T is a simple double array - * @param V such that A = USV^T, V is a 2d array matrix that holds - * singular vectors in columns. Columns are inner arrays - * i.e. V(i)(j) is standard math notation V_{ij} - */ -case class TallSkinnyMatrixSVD(val U: TallSkinnyDenseMatrix, - val S: Array[Double], - val V: Array[Array[Double]]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala index 3218033047e7f..a43ac3635a6e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala @@ -79,7 +79,6 @@ class RowRDDMatrix( RowRDDMatrix.triuToFull(n, GU.data) } - /** * Computes the singular value decomposition of this matrix. * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala deleted file mode 100644 index 3d64ad68c77d4..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.util - -import org.apache.spark.SparkContext._ - -import org.apache.spark.mllib.linalg._ - -/** - * Helper methods for linear algebra - */ -object LAUtils { - /** - * Convert a SparseMatrix into a TallSkinnyDenseMatrix - * - * @param sp Sparse matrix to be converted - * @return dense version of the input - */ - def sparseToTallSkinnyDense(sp: SparseMatrix): TallSkinnyDenseMatrix = { - val m = sp.m - val n = sp.n - val rows = sp.data.map(x => (x.i, (x.j, x.value))).groupByKey().map { - case (i, cols) => - val rowArray = Array.ofDim[Double](n) - var j = 0 - while (j < cols.size) { - rowArray(cols(j)._1) = cols(j)._2 - j += 1 - } - MatrixRow(i, rowArray) - } - TallSkinnyDenseMatrix(rows, m, n) - } - - /** - * Convert a TallSkinnyDenseMatrix to a SparseMatrix - * - * @param a matrix to be converted - * @return sparse version of the input - */ - def denseToSparse(a: TallSkinnyDenseMatrix): SparseMatrix = { - val m = a.m - val n = a.n - val data = a.rows.flatMap { - mrow => Array.tabulate(n)(j => MatrixEntry(mrow.i, j, mrow.data(j))) - .filter(x => x.value != 0) - } - SparseMatrix(data, m, n) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala deleted file mode 100644 index 39ea3389a58d7..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class PCASuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-3 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val ret = DoubleMatrix.zeros(matrix.m, matrix.n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.value)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3)), 3) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("sparse matrix full rank matrix pca") { - val m = 5 - val n = 3 - // the entry that gets dropped is zero to test sparse support - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten.drop(1) - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("truncated matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), - (1,0,-0.1448), (1,1,0.7483), - (2,0,0.9553), (2,1,-0.0649)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val k = 2 - val coeffs = new DoubleMatrix(new PCA().setK(k).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,k)), coeffs) - } -} - - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala deleted file mode 100644 index f9026437529bb..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ /dev/null @@ -1,194 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class SVDSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-4 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - val ret = DoubleMatrix.zeros(m, n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.value)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(n).compute(a) - val u = decomposed.U - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("dense full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val decomposed = new SVD().setK(n).setComputeU(true).compute(a) - val u = LAUtils.denseToSparse(decomposed.U) - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(LAUtils.denseToSparse(a)) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = DoubleMatrix.diag(new DoubleMatrix(s)) - val retv = new DoubleMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("rank one matrix svd") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m, n){ (a,b) => - MatrixEntry(a, b, 1.0) }.flatten ) - val k = 1 - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - assert(retrank == 1, "rank returned not one") - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("truncated with k") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1)/(1 + a + b)) }.flatten ) - val a = SparseMatrix(data, m, n) - - val k = 1 // only one svalue above this - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - assert(retrank == 1, "rank returned not one") - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - } -} From a85262a0d153dd98256ce129d2e03c79855abf4d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 13:36:02 -0700 Subject: [PATCH 04/12] rename RDDMatrixRow to IndexedRDDMatrixRow --- .../apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala | 2 +- .../rdd/{RDDMatrixRow.scala => IndexedRDDMatrixRow.scala} | 2 +- .../apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/{RDDMatrixRow.scala => IndexedRDDMatrixRow.scala} (93%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala index d8a72aed37dda..c6af5fb833b05 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala @@ -66,7 +66,7 @@ class CoordinateRDDMatrix( val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) .groupByKey() .map { case (i, vectorEntries) => - RDDMatrixRow(i, Vectors.sparse(n, vectorEntries)) + IndexedRDDMatrixRow(i, Vectors.sparse(n, vectorEntries)) } new IndexedRowRDDMatrix(indexedRows, numRows(), numCols()) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRDDMatrixRow.scala similarity index 93% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixRow.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRDDMatrixRow.scala index b964fc206f9d3..8c9a9a857646c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixRow.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRDDMatrixRow.scala @@ -20,4 +20,4 @@ package org.apache.spark.mllib.linalg.rdd import org.apache.spark.mllib.linalg.Vector /** Represents a row of RowRDDMatrix. */ -case class RDDMatrixRow(index: Long, vector: Vector) +case class IndexedRDDMatrixRow(index: Long, vector: Vector) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala index c0ead63a84c7e..d1dbd0a017c2d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala @@ -27,7 +27,7 @@ import org.apache.spark.rdd.RDD * @param n number of cols, where a negative number means unknown */ class IndexedRowRDDMatrix( - val rows: RDD[RDDMatrixRow], + val rows: RDD[IndexedRDDMatrixRow], m: Long = -1L, n: Long = -1L) extends RDDMatrix { @@ -49,8 +49,8 @@ class IndexedRowRDDMatrix( _m } - /** Drops row indices and converts this to a RowRDDMatrix. */ - def compressRows(): RowRDDMatrix = { + /** Drops row indices and converts this matrix to a RowRDDMatrix. */ + def toRowRDDMatrix(): RowRDDMatrix = { new RowRDDMatrix(rows.map(_.vector), -1, _n) } } From 0d1491cd6f76ea0a1c773bb2f482de756459a749 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 13:51:49 -0700 Subject: [PATCH 05/12] fix test errors --- .../spark/mllib/linalg/rdd/RowRDDMatrix.scala | 15 ++++++++------- .../linalg/rdd/CoordinateRDDMatrixSuite.scala | 6 +++--- .../mllib/linalg/rdd/RowRDDMatrixSuite.scala | 3 ++- 3 files changed, 13 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala index a43ac3635a6e7..076abf4dd8d4c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala @@ -139,17 +139,18 @@ class RowRDDMatrix( val V = Matrices.dense(n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) if (computeU) { - val N = new BDM[Double](sk, n, util.Arrays.copyOfRange(v.data, 0, sk * n)) + // N = Vk * Sk^{-1} + val N = new BDM[Double](n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) var i = 0 var j = 0 - while (i < sk) { - j = 0 - val sigma = sigmas(i) - while (j < n) { + while (j < sk) { + i = 0 + val sigma = sigmas(j) + while (i < n) { N(i, j) /= sigma - j += 1 + i += 1 } - i += 1 + j += 1 } val U = this.multiply(Matrices.fromBreeze(N)) SingularValueDecomposition(U, s, V) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala index 7911c4103e434..5a5af5106b510 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.mllib.linalg.Vectors class CoordinateRDDMatrixSuite extends FunSuite with LocalSparkContext { - val m = 4 + val m = 5 val n = 4 var mat: CoordinateRDDMatrix = _ @@ -46,8 +46,8 @@ class CoordinateRDDMatrixSuite extends FunSuite with LocalSparkContext { } test("size") { - assert(mat.numRows() === 4) - assert(mat.numCols() === 5) + assert(mat.numRows() === m) + assert(mat.numCols() === n) } test("toIndexedRowRDDMatrix") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala index 2b85481df4d6d..c6bd2fbcc0e2a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala @@ -80,7 +80,8 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { val brzV = svd.V.toBreeze.asInstanceOf[BDM[Double]] val rows = U.rows.collect() val brzUt = new BDM[Double](n, m, rows.flatMap(r => r.toArray)) - assert(closeToZero(brzUt.t * brzDiag(brzSigma) * brzV.t - A)) + val UsVt = brzUt.t * brzDiag(brzSigma) * brzV.t + assert(closeToZero(UsVt - A)) val VtV: BDM[Double] = brzV.t * brzV assert(closeToZero(VtV - BDM.eye[Double](n))) val UtU = U.computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] From e7d0d4ac5a8eafdd086d3ac49ee2fdbf8bc3644c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 13:53:28 -0700 Subject: [PATCH 06/12] move IndexedRDDMatrixRow to IndexedRowRDDMatrix --- .../linalg/rdd/IndexedRDDMatrixRow.scala | 23 ------------------- .../linalg/rdd/IndexedRowRDDMatrix.scala | 4 ++++ 2 files changed, 4 insertions(+), 23 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRDDMatrixRow.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRDDMatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRDDMatrixRow.scala deleted file mode 100644 index 8c9a9a857646c..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRDDMatrixRow.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg.rdd - -import org.apache.spark.mllib.linalg.Vector - -/** Represents a row of RowRDDMatrix. */ -case class IndexedRDDMatrixRow(index: Long, vector: Vector) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala index d1dbd0a017c2d..c3e50f1160175 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala @@ -18,6 +18,10 @@ package org.apache.spark.mllib.linalg.rdd import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector + +/** Represents a row of RowRDDMatrix. */ +case class IndexedRDDMatrixRow(index: Long, vector: Vector) /** * Represents a row-oriented RDDMatrix with indexed rows. From b881506c8237bb298b9c63208935ddb2ec97f78a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 6 Apr 2014 16:23:58 -0700 Subject: [PATCH 07/12] rename SparkPCA/SVD to TallSkinnyPCA/SVD --- .../spark/examples/mllib/SparkPCA.scala | 48 -------------- .../spark/examples/mllib/SparkSVD.scala | 57 ----------------- .../spark/examples/mllib/TallSkinnyPCA.scala | 64 +++++++++++++++++++ .../spark/examples/mllib/TallSkinnySVD.scala | 64 +++++++++++++++++++ 4 files changed, 128 insertions(+), 105 deletions(-) delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala deleted file mode 100644 index 85fc5b3a08e12..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.util._ - - -/** - * Compute PCA of an example matrix. - */ -object SparkPCA { - def main(args: Array[String]) { - if (args.length != 3) { - System.err.println("Usage: SparkPCA m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "PCA", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - val m = args(2).toInt - val n = args(3).toInt - - // Make example matrix - val data = Array.tabulate(m, n) { (a, b) => - (a + 2).toDouble * (b + 1) / (1 + a + b) } - - // recover top principal component - val coeffs = new PCA().setK(1).compute(sc.makeRDD(data)) - - println("top principal component = " + coeffs.mkString(", ")) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala deleted file mode 100644 index 8068c0da96920..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.SVD - -/** - * Compute SVD of an example matrix - * Input file should be comma separated, 1 indexed of the form - * i,j,value - * Where i is the column, j the row, and value is the matrix entry - * - * For example input file, see: - * mllib/data/als/test.data (example is 4 x 4) - */ -object SparkSVD { - def main(args: Array[String]) { - if (args.length != 4) { - System.err.println("Usage: SparkSVD m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "SVD", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - // Load and parse the data file - val data = sc.textFile(args(1)).map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt - 1, parts(1).toInt - 1, parts(2).toDouble) - } - val m = args(2).toInt - val n = args(3).toInt - - // recover largest singular vector - val decomposed = new SVD().setK(1).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("singular values = " + s.collect().mkString) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala new file mode 100644 index 0000000000000..a0945761b5e15 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.rdd.RowRDDMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the principal components of a tall-and-skinny matrix, whose rows are observations. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0 1 + * 2 3 + * 4 5 + * }}} + * represents a 3-by-2 matrix, whose first row is (0, 1). + */ +object TallSkinnyPCA { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnyPCA ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnyPCA") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowRDDMatrix(rows) + + // Compute principal components. + val pc = mat.computePrincipalComponents(mat.numCols().toInt) + + println("Principal components are:\n" + pc) + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala new file mode 100644 index 0000000000000..0f7825e10defa --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.rdd.RowRDDMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the singular value decomposition (SVD) of a tall-and-skinny matrix. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0 1 + * 2 3 + * 4 5 + * }}} + * represents a 3-by-2 matrix, whose first row is (0, 1). + */ +object TallSkinnySVD { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnySVD ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnySVD") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowRDDMatrix(rows) + + // Compute SVD. + val svd = mat.computeSVD(mat.numCols().toInt) + + println("Singular values are " + svd.s) + + sc.stop() + } +} From be119fea96b4ef0cd2fa49cbe122dc733e777e74 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 6 Apr 2014 17:03:36 -0700 Subject: [PATCH 08/12] rename m/n to numRows/numCols for local matrix add tests for matrices --- .../apache/spark/mllib/linalg/Matrices.scala | 42 +++++++++++++------ .../spark/mllib/linalg/rdd/RowRDDMatrix.scala | 4 +- .../linalg/BreezeMatrixConversionSuite.scala | 40 ++++++++++++++++++ .../spark/mllib/linalg/MatricesSuite.scala | 39 +++++++++++++++++ .../mllib/linalg/rdd/RowRDDMatrixSuite.scala | 4 +- 5 files changed, 112 insertions(+), 17 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala 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 5ecf7c163a823..a0365338da0c9 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 @@ -25,44 +25,60 @@ import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} trait Matrix extends Serializable { /** Number of rows. */ - def m: Int + def numRows: Int /** Number of columns. */ - def n: Int + def numCols: Int /** Converts to a dense array in column major. */ def toArray: Array[Double] /** Converts to a breeze matrix. */ private[mllib] def toBreeze: BM[Double] + + /** Gets the (i, j)-th element. */ + private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) } /** - * Column majored dense matrix. + * Column-majored dense matrix. * - * @param m - * @param n - * @param values + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major */ -class DenseMatrix(val m: Int, val n: Int, val values: Array[Double]) extends Matrix { +class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) extends Matrix { - require(values.length == m * n) + require(values.length == numRows * numCols) - def toArray: Array[Double] = values + override def toArray: Array[Double] = values - private[mllib] def toBreeze: BM[Double] = new BDM[Double](m, n, values) + private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) } object Matrices { - def dense(m: Int, n: Int, values: Array[Double]): Matrix = { - new DenseMatrix(m, n, values) + /** + * Creates a dense matrix. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ + def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { + new DenseMatrix(numRows, numCols, values) } + /** + * Creates a Matrix instance from a breeze matrix. + * @param breeze a breeze matrix + * @return a Matrix instance + */ private[mllib] def fromBreeze(breeze: BM[Double]): Matrix = { breeze match { case dm: BDM[Double] => - require(dm.majorStride == dm.rows) + require(dm.majorStride == dm.rows, + "Do not support stride size different from the number of rows.") new DenseMatrix(dm.rows, dm.cols, dm.data) case _ => throw new UnsupportedOperationException( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala index 076abf4dd8d4c..9d64fbea68e89 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala @@ -243,7 +243,7 @@ class RowRDDMatrix( */ def multiply(B: Matrix): RowRDDMatrix = { val n = numCols().toInt - require(n == B.m, s"Dimension mismatch: $n vs ${B.m}") + require(n == B.numRows, s"Dimension mismatch: $n vs ${B.numRows}") require(B.isInstanceOf[DenseMatrix], s"Only support dense matrix at this time but found ${B.getClass.getName}.") @@ -254,7 +254,7 @@ class RowRDDMatrix( iter.map(v => Vectors.fromBreeze(Bi.t * v.toBreeze)) }, preservesPartitioning = true) - new RowRDDMatrix(AB, _m, B.n) + new RowRDDMatrix(AB, _m, B.numCols) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala new file mode 100644 index 0000000000000..82d49c76ed02b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +class BreezeMatrixConversionSuite extends FunSuite { + test("dense matrix to breeze") { + val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val breeze = mat.toBreeze.asInstanceOf[BDM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") + } + + test("dense breeze matrix to matrix") { + val breeze = new BDM[Double](3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[DenseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala new file mode 100644 index 0000000000000..9c66b4db9f16b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import org.scalatest.FunSuite + +class MatricesSuite extends FunSuite { + test("dense matrix construction") { + val m = 3 + val n = 2 + val values = Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0) + val mat = Matrices.dense(m, n, values).asInstanceOf[DenseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.toArray.eq(values), "toArray should not copy data") + } + + test("dense matrix construction with wrong dimension") { + intercept[RuntimeException] { + Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala index c6bd2fbcc0e2a..17ea149a0862e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala @@ -112,8 +112,8 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { test("pca") { for (mat <- Seq(denseMat, sparseMat); k <- 1 to n) { val pc = denseMat.computePrincipalComponents(k) - assert(pc.m === n) - assert(pc.n === k) + assert(pc.numRows === n) + assert(pc.numCols === k) assertPrincipalComponentsEqual(pc, principalComponents, k) } } From b177ff1035ec4ac4cd096ae70539d2805a1ad8a4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 7 Apr 2014 15:53:10 -0700 Subject: [PATCH 09/12] address Matei's comments --- .../spark/examples/mllib/TallSkinnyPCA.scala | 4 +- .../spark/examples/mllib/TallSkinnySVD.scala | 4 +- .../apache/spark/mllib/linalg/Matrices.scala | 7 +- .../CoordinateMatrix.scala} | 50 ++++++++------ .../DistributedMatrix.scala} | 6 +- .../DistributedMatrixEntry.scala} | 4 +- .../IndexedRowMatrix.scala} | 40 ++++++----- .../RowMatrix.scala} | 68 +++++++++---------- .../CoordinateMatrixSuite.scala} | 25 +++++-- .../RowMatrixSuite.scala} | 25 +++++-- 10 files changed, 134 insertions(+), 99 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{rdd/CoordinateRDDMatrix.scala => distributed/CoordinateMatrix.scala} (55%) rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{rdd/RDDMatrix.scala => distributed/DistributedMatrix.scala} (84%) rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{rdd/RDDMatrixEntry.scala => distributed/DistributedMatrixEntry.scala} (88%) rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{rdd/IndexedRowRDDMatrix.scala => distributed/IndexedRowMatrix.scala} (53%) rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{rdd/RowRDDMatrix.scala => distributed/RowMatrix.scala} (86%) rename mllib/src/test/scala/org/apache/spark/mllib/linalg/{rdd/CoordinateRDDMatrixSuite.scala => distributed/CoordinateMatrixSuite.scala} (75%) rename mllib/src/test/scala/org/apache/spark/mllib/linalg/{rdd/RowRDDMatrixSuite.scala => distributed/RowMatrixSuite.scala} (87%) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index a0945761b5e15..4d5560c5350b2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.linalg.rdd.RowRDDMatrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors /** @@ -52,7 +52,7 @@ object TallSkinnyPCA { val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } - val mat = new RowRDDMatrix(rows) + val mat = new RowMatrix(rows) // Compute principal components. val pc = mat.computePrincipalComponents(mat.numCols().toInt) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 0f7825e10defa..0b92d7c934207 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.linalg.rdd.RowRDDMatrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors /** @@ -52,7 +52,7 @@ object TallSkinnySVD { val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } - val mat = new RowRDDMatrix(rows) + val mat = new RowMatrix(rows) // Compute SVD. val svd = mat.computeSVD(mat.numCols().toInt) 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 a0365338da0c9..51b2b0bcfd3c6 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 @@ -20,7 +20,7 @@ package org.apache.spark.mllib.linalg import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} /** - * Trait for matrix. + * Trait for a local matrix. */ trait Matrix extends Serializable { @@ -56,10 +56,13 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) } +/** + * Factory methods for [[org.apache.spark.mllib.linalg.Matrix]]. + */ object Matrices { /** - * Creates a dense matrix. + * Creates a column-majored dense matrix. * * @param numRows number of rows * @param numCols number of columns diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala similarity index 55% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index c6af5fb833b05..09825a4d1e923 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -15,59 +15,67 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.linalg.distributed import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vectors /** - * Represents a matrix in coordinate list format. + * Represents a matrix in coordinate format. * * @param entries matrix entries - * @param m number of rows (default: -1L, which means unknown) - * @param n number of column (default: -1L, which means unknown) + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the max column index plus one. */ -class CoordinateRDDMatrix( - val entries: RDD[RDDMatrixEntry], - m: Long = -1L, - n: Long = -1L) extends RDDMatrix { +class CoordinateMatrix( + val entries: RDD[DistributedMatrixEntry], + private var nRows: Long, + private var nCols: Long) extends DistributedMatrix { - private var _m = m - private var _n = n + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(entries: RDD[DistributedMatrixEntry]) = this(entries, 0L, 0L) /** Gets or computes the number of columns. */ override def numCols(): Long = { - if (_n < 0) { + if (nCols <= 0L) { computeSize() } - _n + nCols } /** Gets or computes the number of rows. */ override def numRows(): Long = { - if (_m < 0) { + if (nRows <= 0L) { computeSize() } - _m + nRows } private def computeSize() { + // Reduce will throw an exception if `entries` is empty. val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => (math.max(i1, i2), math.max(j1, j2)) } // There may be empty columns at the very right and empty rows at the very bottom. - _m = math.max(_m, m1 + 1L) - _n = math.max(_n, n1 + 1L) + nRows = math.max(nRows, m1 + 1L) + nCols = math.max(nCols, n1 + 1L) } - def toIndexedRowRDDMatrix(): IndexedRowRDDMatrix = { - val n = numCols().toInt + def toIndexedRowMatrix(): IndexedRowMatrix = { + val nl = numCols() + if (nl > Int.MaxValue) { + sys.error(s"Cannot convert to a row-oriented format because the number of columns $nl is " + + "too large.") + } + val n = nl.toInt val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) .groupByKey() .map { case (i, vectorEntries) => - IndexedRDDMatrixRow(i, Vectors.sparse(n, vectorEntries)) - } - new IndexedRowRDDMatrix(indexedRows, numRows(), numCols()) + IndexedMatrixRow(i, Vectors.sparse(n, vectorEntries)) + } + new IndexedRowMatrix(indexedRows, numRows(), n) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala similarity index 84% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrix.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index 9452593ed091f..03ce8b55c5d53 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.linalg.distributed /** - * Represents a matrix backed by one or more RDDs. + * Represents a distributively stored matrix backed by one or more RDDs. */ -trait RDDMatrix extends Serializable { +trait DistributedMatrix extends Serializable { /** Gets or computes the number of rows. */ def numRows(): Long diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixEntry.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrixEntry.scala similarity index 88% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixEntry.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrixEntry.scala index 79dadf1fbb1fe..ab568b45151bf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RDDMatrixEntry.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrixEntry.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.linalg.distributed /** * Represents an entry in an RDDMatrix. @@ -23,4 +23,4 @@ package org.apache.spark.mllib.linalg.rdd * @param j column index * @param value value of the entry */ -case class RDDMatrixEntry(i: Long, j: Long, value: Double) +case class DistributedMatrixEntry(i: Long, j: Long, value: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala similarity index 53% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index c3e50f1160175..01ca00a4429ea 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/IndexedRowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -15,46 +15,50 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.linalg.distributed import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector /** Represents a row of RowRDDMatrix. */ -case class IndexedRDDMatrixRow(index: Long, vector: Vector) +case class IndexedMatrixRow(index: Long, vector: Vector) /** * Represents a row-oriented RDDMatrix with indexed rows. * * @param rows indexed rows of this matrix - * @param m number of rows, where a negative number means unknown - * @param n number of cols, where a negative number means unknown + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. */ -class IndexedRowRDDMatrix( - val rows: RDD[IndexedRDDMatrixRow], - m: Long = -1L, - n: Long = -1L) extends RDDMatrix { +class IndexedRowMatrix( + val rows: RDD[IndexedMatrixRow], + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix { - private var _m = m - private var _n = n + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[IndexedMatrixRow]) = this(rows, 0L, 0) /** Gets or computes the number of columns. */ override def numCols(): Long = { - if (_n < 0) { - _n = rows.first().vector.size + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().vector.size } - _n + nCols } override def numRows(): Long = { - if (_m < 0) { - _m = rows.map(_.index).reduce(math.max) + 1 + if (nRows <= 0L) { + // Reduce will throw an exception if `rows` is empty. + nRows = rows.map(_.index).reduce(math.max) + 1L } - _m + nRows } /** Drops row indices and converts this matrix to a RowRDDMatrix. */ - def toRowRDDMatrix(): RowRDDMatrix = { - new RowRDDMatrix(rows.map(_.vector), -1, _n) + def toRowMatrix(): RowMatrix = { + new RowMatrix(rows.map(_.vector), 0L, nCols) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala similarity index 86% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 9d64fbea68e89..2c13feba8bc28 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.linalg.distributed import java.util -import scala.util.control.Breaks._ - import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -33,31 +31,38 @@ import org.apache.spark.Logging * Represents a row-oriented RDDMatrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] - * @param m number of rows - * @param n number of columns + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the number of records in the RDD `rows`. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. */ -class RowRDDMatrix( +class RowMatrix( val rows: RDD[Vector], - m: Long = -1L, - n: Long = -1) extends RDDMatrix with Logging { + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix with Logging { - private var _m = m - private var _n = n + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[Vector]) = this(rows, 0L, 0) /** Gets or computes the number of columns. */ override def numCols(): Long = { - if (_n < 0) { - _n = rows.first().size + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().size } - _n + nCols } /** Gets or computes the number of rows. */ override def numRows(): Long = { - if (_m < 0) { - _m = rows.count() + if (nRows <= 0L) { + nRows = rows.count() + if (nRows == 0L) { + sys.error("Cannot determine the number of rows because it is not specified in the " + + "constructor and the rows RDD is empty.") + } } - _m + nRows } /** @@ -70,13 +75,13 @@ class RowRDDMatrix( // Compute the upper triangular part of the gram matrix. val GU = rows.aggregate(new BDV[Double](new Array[Double](nt)))( seqOp = (U, v) => { - RowRDDMatrix.dspr(1.0, v, U.data) + RowMatrix.dspr(1.0, v, U.data) U }, combOp = (U1, U2) => U1 += U2 ) - RowRDDMatrix.triuToFull(n, GU.data) + RowMatrix.triuToFull(n, GU.data) } /** @@ -108,10 +113,8 @@ class RowRDDMatrix( def computeSVD( k: Int, computeU: Boolean = false, - rCond: Double = 1e-9): SingularValueDecomposition[RowRDDMatrix, Matrix] = { - + rCond: Double = 1e-9): SingularValueDecomposition[RowMatrix, Matrix] = { val n = numCols().toInt - require(k > 0 && k <= n, s"Request up to n singular values k=$k n=$n.") val G = computeGramianMatrix() @@ -125,13 +128,8 @@ class RowRDDMatrix( val sigma0 = sigmas(0) val threshold = rCond * sigma0 var i = 0 - breakable { - while (i < k) { - if (sigmas(i) < threshold) { - break() - } - i += 1 - } + while (i < k && sigmas(i) >= threshold) { + i += 1 } val sk = i @@ -178,11 +176,11 @@ class RowRDDMatrix( ) // Update _m if it is not set, or verify its value. - if (_m < 0L) { - _m = m + if (nRows <= 0L) { + nRows = m } else { - require(_m == m, - s"The number of rows $m is different from what specified or previously computed: ${_m}.") + require(nRows == m, + s"The number of rows $m is different from what specified or previously computed: ${nRows}.") } mean :/= m.toDouble @@ -241,7 +239,7 @@ class RowRDDMatrix( * @param B a local matrix whose number of rows must match the number of columns of this matrix * @return a RowRDDMatrix representing the product, which preserves partitioning */ - def multiply(B: Matrix): RowRDDMatrix = { + def multiply(B: Matrix): RowMatrix = { val n = numCols().toInt require(n == B.numRows, s"Dimension mismatch: $n vs ${B.numRows}") @@ -254,11 +252,11 @@ class RowRDDMatrix( iter.map(v => Vectors.fromBreeze(Bi.t * v.toBreeze)) }, preservesPartitioning = true) - new RowRDDMatrix(AB, _m, B.numCols) + new RowMatrix(AB, nRows, B.numCols) } } -object RowRDDMatrix { +object RowMatrix { /** * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala similarity index 75% rename from mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala index 5a5af5106b510..5d07840d6d273 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/CoordinateRDDMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.linalg.distributed import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors -class CoordinateRDDMatrixSuite extends FunSuite with LocalSparkContext { +class CoordinateMatrixSuite extends FunSuite with LocalSparkContext { val m = 5 val n = 4 - var mat: CoordinateRDDMatrix = _ + var mat: CoordinateMatrix = _ override def beforeAll() { super.beforeAll() @@ -40,9 +40,9 @@ class CoordinateRDDMatrixSuite extends FunSuite with LocalSparkContext { (3, 0, 7.0), (3, 3, 8.0), (4, 1, 9.0)), 3).map { case (i, j, value) => - RDDMatrixEntry(i, j, value) + DistributedMatrixEntry(i, j, value) } - mat = new CoordinateRDDMatrix(entries) + mat = new CoordinateMatrix(entries) } test("size") { @@ -50,9 +50,20 @@ class CoordinateRDDMatrixSuite extends FunSuite with LocalSparkContext { assert(mat.numCols() === n) } - test("toIndexedRowRDDMatrix") { + test("empty entries") { + val entries = sc.parallelize(Seq[DistributedMatrixEntry](), 1) + val emptyMat = new CoordinateMatrix(entries) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + + test("toIndexedRowMatrix") { val indexedRows = mat - .toIndexedRowRDDMatrix() + .toIndexedRowMatrix() .rows .map(row => (row.index, row.vector)) .collect() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala similarity index 87% rename from mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 17ea149a0862e..6422d4e15565d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/RowRDDMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.linalg.distributed import org.scalatest.FunSuite import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, diag => brzDiag, norm => brzNorm} import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.linalg.{Matrices, Vectors, Matrix} +import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector, Matrix} -class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { +class RowMatrixSuite extends FunSuite with LocalSparkContext { val m = 4 val n = 3 @@ -46,13 +46,13 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { Array(0.0, math.sqrt(2.0) / 2.0, math.sqrt(2.0) / 2.0, 1.0, 0.0, 0.0, 0.0, math.sqrt(2.0) / 2.0, - math.sqrt(2.0) / 2.0)) - var denseMat: RowRDDMatrix = _ - var sparseMat: RowRDDMatrix = _ + var denseMat: RowMatrix = _ + var sparseMat: RowMatrix = _ override def beforeAll() { super.beforeAll() - denseMat = new RowRDDMatrix(sc.parallelize(denseData, 2)) - sparseMat = new RowRDDMatrix(sc.parallelize(sparseData, 2)) + denseMat = new RowMatrix(sc.parallelize(denseData, 2)) + sparseMat = new RowMatrix(sc.parallelize(sparseData, 2)) } test("size") { @@ -62,6 +62,17 @@ class RowRDDMatrixSuite extends FunSuite with LocalSparkContext { assert(sparseMat.numCols() === n) } + test("empty rows") { + val rows = sc.parallelize(Seq[Vector](), 1) + val emptyMat = new RowMatrix(rows) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + test("gram") { val expected = Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) From 03cd7e1824e4315839243412c6ea6663dcc48a03 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 00:14:09 -0700 Subject: [PATCH 10/12] add pca/gram to IndexedRowMatrix add toBreeze to DistributedMatrix for test simplify tests --- .../linalg/distributed/CoordinateMatrix.scala | 57 +++++++-- .../distributed/DistributedMatrix.scala | 7 + .../distributed/DistributedMatrixEntry.scala | 26 ---- .../linalg/distributed/IndexedRowMatrix.scala | 100 +++++++++++++-- .../mllib/linalg/distributed/RowMatrix.scala | 21 ++- .../distributed/CoordinateMatrixSuite.scala | 50 +++++--- .../distributed/IndexedRowMatrixSuite.scala | 120 ++++++++++++++++++ .../linalg/distributed/RowMatrixSuite.scala | 11 ++ 8 files changed, 326 insertions(+), 66 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrixEntry.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 09825a4d1e923..710dbbdae3659 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -17,10 +17,20 @@ package org.apache.spark.mllib.linalg.distributed +import breeze.linalg.{DenseMatrix => BDM} + import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vectors +/** + * Represents an entry in an distributed matrix. + * @param i row index + * @param j column index + * @param value value of the entry + */ +case class MatrixEntry(i: Long, j: Long, value: Double) + /** * Represents a matrix in coordinate format. * @@ -31,12 +41,12 @@ import org.apache.spark.mllib.linalg.Vectors * columns will be determined by the max column index plus one. */ class CoordinateMatrix( - val entries: RDD[DistributedMatrixEntry], + val entries: RDD[MatrixEntry], private var nRows: Long, private var nCols: Long) extends DistributedMatrix { /** Alternative constructor leaving matrix dimensions to be determined automatically. */ - def this(entries: RDD[DistributedMatrixEntry]) = this(entries, 0L, 0L) + def this(entries: RDD[MatrixEntry]) = this(entries, 0L, 0L) /** Gets or computes the number of columns. */ override def numCols(): Long = { @@ -54,16 +64,7 @@ class CoordinateMatrix( nRows } - private def computeSize() { - // Reduce will throw an exception if `entries` is empty. - val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => - (math.max(i1, i2), math.max(j1, j2)) - } - // There may be empty columns at the very right and empty rows at the very bottom. - nRows = math.max(nRows, m1 + 1L) - nCols = math.max(nCols, n1 + 1L) - } - + /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */ def toIndexedRowMatrix(): IndexedRowMatrix = { val nl = numCols() if (nl > Int.MaxValue) { @@ -74,8 +75,38 @@ class CoordinateMatrix( val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) .groupByKey() .map { case (i, vectorEntries) => - IndexedMatrixRow(i, Vectors.sparse(n, vectorEntries)) + IndexedRow(i, Vectors.sparse(n, vectorEntries)) } new IndexedRowMatrix(indexedRows, numRows(), n) } + + /** + * Converts to RowMatrix, dropping row indices after grouping by row index. + * The number of columns must be within the integer range. + */ + def toRowMatrix(): RowMatrix = { + toIndexedRowMatrix().toRowMatrix() + } + + /** Determines the size by computing the max row/column index. */ + private def computeSize() { + // Reduce will throw an exception if `entries` is empty. + val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => + (math.max(i1, i2), math.max(j1, j2)) + } + // There may be empty columns at the very right and empty rows at the very bottom. + nRows = math.max(nRows, m1 + 1L) + nCols = math.max(nCols, n1 + 1L) + } + + /** Collects data and assembles a local matrix. */ + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + entries.collect().foreach { case MatrixEntry(i, j, value) => + mat(i.toInt, j.toInt) = value + } + mat + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index 03ce8b55c5d53..13f72a3c724ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -17,6 +17,10 @@ package org.apache.spark.mllib.linalg.distributed +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.mllib.linalg.Matrix + /** * Represents a distributively stored matrix backed by one or more RDDs. */ @@ -27,4 +31,7 @@ trait DistributedMatrix extends Serializable { /** Gets or computes the number of columns. */ def numCols(): Long + + /** Collects data and assembles a local dense breeze matrix (for test only). */ + private[mllib] def toBreeze(): BDM[Double] } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrixEntry.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrixEntry.scala deleted file mode 100644 index ab568b45151bf..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrixEntry.scala +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg.distributed - -/** - * Represents an entry in an RDDMatrix. - * @param i row index - * @param j column index - * @param value value of the entry - */ -case class DistributedMatrixEntry(i: Long, j: Long, value: Double) 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 01ca00a4429ea..e110f070bd7c1 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 @@ -17,14 +17,18 @@ package org.apache.spark.mllib.linalg.distributed +import breeze.linalg.{DenseMatrix => BDM} + import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.linalg.SingularValueDecomposition -/** Represents a row of RowRDDMatrix. */ -case class IndexedMatrixRow(index: Long, vector: Vector) +/** Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ +case class IndexedRow(index: Long, vector: Vector) /** - * Represents a row-oriented RDDMatrix with indexed rows. + * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with + * indexed rows. * * @param rows indexed rows of this matrix * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will @@ -33,14 +37,13 @@ case class IndexedMatrixRow(index: Long, vector: Vector) * columns will be determined by the size of the first row. */ class IndexedRowMatrix( - val rows: RDD[IndexedMatrixRow], + val rows: RDD[IndexedRow], private var nRows: Long, private var nCols: Int) extends DistributedMatrix { /** Alternative constructor leaving matrix dimensions to be determined automatically. */ - def this(rows: RDD[IndexedMatrixRow]) = this(rows, 0L, 0) + def this(rows: RDD[IndexedRow]) = this(rows, 0L, 0) - /** Gets or computes the number of columns. */ override def numCols(): Long = { if (nCols <= 0) { // Calling `first` will throw an exception if `rows` is empty. @@ -57,8 +60,89 @@ class IndexedRowMatrix( nRows } - /** Drops row indices and converts this matrix to a RowRDDMatrix. */ + /** + * Drops row indices and converts this matrix to a + * [[org.apache.spark.mllib.linalg.distributed.RowMatrix]]. + */ def toRowMatrix(): RowMatrix = { new RowMatrix(rows.map(_.vector), 0L, nCols) } + + /** + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * + * There is no restriction on m, but we require `n^2` doubles to fit in memory. + * Further, n should be less than m. + + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. + * + * At most k largest non-zero singular values and associated vectors are returned. + * If there are k such values, then the dimensions of the return will be: + * + * U is an [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]] of size m x k that + * satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a local Matrix of size n x k that satisfies V'V = eye(k). + * + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[IndexedRowMatrix, Matrix] = { + val indices = rows.map(_.index) + val svd = toRowMatrix().computeSVD(k, computeU, rCond) + val U = if (computeU) { + val indexedRows = indices.zip(svd.U.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } else { + null + } + SingularValueDecomposition(U, svd.s, svd.V) + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return an IndexedRowMatrix representing the product, which preserves partitioning + */ + def multiply(B: Matrix): IndexedRowMatrix = { + val mat = toRowMatrix().multiply(B) + val indexedRows = rows.map(_.index).zip(mat.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } + + /** + * Computes the Gramian matrix `A^T A`. + */ + def computeGramianMatrix(): Matrix = { + toRowMatrix().computeGramianMatrix() + } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + rows.collect().foreach { case IndexedRow(rowIndex, vector) => + val i = rowIndex.toInt + vector.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + } + mat + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 2c13feba8bc28..24f6b72632b2d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -28,7 +28,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.Logging /** - * Represents a row-oriented RDDMatrix with no meaningful row indices. + * Represents a row-oriented distributed Matrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will @@ -99,7 +99,7 @@ class RowMatrix( * At most k largest non-zero singular values and associated vectors are returned. * If there are k such values, then the dimensions of the return will be: * - * U is a RowRDDMatrix of size m x k that satisfies U'U = eye(k), + * U is a RowMatrix of size m x k that satisfies U'U = eye(k), * s is a Vector of size k, holding the singular values in descending order, * and V is a Matrix of size n x k that satisfies V'V = eye(k). * @@ -237,7 +237,8 @@ class RowMatrix( * Multiply this matrix by a local matrix on the right. * * @param B a local matrix whose number of rows must match the number of columns of this matrix - * @return a RowRDDMatrix representing the product, which preserves partitioning + * @return a [[org.apache.spark.mllib.linalg.distributed.RowMatrix]] representing the product, + * which preserves partitioning */ def multiply(B: Matrix): RowMatrix = { val n = numCols().toInt @@ -254,6 +255,20 @@ class RowMatrix( new RowMatrix(AB, nRows, B.numCols) } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + var i = 0 + rows.collect().foreach { v => + v.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + i += 1 + } + mat + } } object RowMatrix { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala index 5d07840d6d273..cd45438fb628f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.linalg.distributed import org.scalatest.FunSuite +import breeze.linalg.{DenseMatrix => BDM} + import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors @@ -40,7 +42,7 @@ class CoordinateMatrixSuite extends FunSuite with LocalSparkContext { (3, 0, 7.0), (3, 3, 8.0), (4, 1, 9.0)), 3).map { case (i, j, value) => - DistributedMatrixEntry(i, j, value) + MatrixEntry(i, j, value) } mat = new CoordinateMatrix(entries) } @@ -51,7 +53,7 @@ class CoordinateMatrixSuite extends FunSuite with LocalSparkContext { } test("empty entries") { - val entries = sc.parallelize(Seq[DistributedMatrixEntry](), 1) + val entries = sc.parallelize(Seq[MatrixEntry](), 1) val emptyMat = new CoordinateMatrix(entries) intercept[RuntimeException] { emptyMat.numCols() @@ -61,20 +63,36 @@ class CoordinateMatrixSuite extends FunSuite with LocalSparkContext { } } + test("toBreeze") { + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(mat.toBreeze() === expected) + } + test("toIndexedRowMatrix") { - val indexedRows = mat - .toIndexedRowMatrix() - .rows - .map(row => (row.index, row.vector)) - .collect() - .sortBy(_._1) - .toSeq - assert(indexedRows === Seq( - (0, Vectors.dense(1.0, 2.0, 0.0, 0.0)), - (1, Vectors.dense(0.0, 3.0, 4.0, 0.0)), - (2, Vectors.dense(0.0, 0.0, 5.0, 6.0)), - (3, Vectors.dense(7.0, 0.0, 0.0, 8.0)), - (4, Vectors.dense(0.0, 9.0, 0.0, 0.0)) - )) + val indexedRowMatrix = mat.toIndexedRowMatrix() + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(indexedRowMatrix.toBreeze() === expected) + } + + test("toRowMatrix") { + val rowMatrix = mat.toRowMatrix() + val rows = rowMatrix.rows.collect().toSet + val expected = Set( + Vectors.dense(1.0, 2.0, 0.0, 0.0), + Vectors.dense(0.0, 3.0, 4.0, 0.0), + Vectors.dense(0.0, 0.0, 5.0, 6.0), + Vectors.dense(7.0, 0.0, 0.0, 8.0), + Vectors.dense(0.0, 9.0, 0.0, 0.0)) + assert(rows === expected) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala new file mode 100644 index 0000000000000..f7c46f23b746d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{diag => brzDiag, DenseMatrix => BDM, DenseVector => BDV} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrices, Vectors} + +class IndexedRowMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val data = Seq( + (0L, Vectors.dense(0.0, 1.0, 2.0)), + (1L, Vectors.dense(3.0, 4.0, 5.0)), + (3L, Vectors.dense(9.0, 0.0, 1.0)) + ).map(x => IndexedRow(x._1, x._2)) + var indexedRows: RDD[IndexedRow] = _ + + override def beforeAll() { + super.beforeAll() + indexedRows = sc.parallelize(data, 2) + } + + test("size") { + val mat1 = new IndexedRowMatrix(indexedRows) + assert(mat1.numRows() === m) + assert(mat1.numCols() === n) + + val mat2 = new IndexedRowMatrix(indexedRows, 5, 0) + assert(mat2.numRows() === 5) + assert(mat2.numCols() === n) + } + + test("empty rows") { + val rows = sc.parallelize(Seq[IndexedRow](), 1) + val mat = new IndexedRowMatrix(rows) + intercept[RuntimeException] { + mat.numRows() + } + intercept[RuntimeException] { + mat.numCols() + } + } + + test("toBreeze") { + val mat = new IndexedRowMatrix(indexedRows) + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (0.0, 0.0, 0.0), + (9.0, 0.0, 1.0)) + assert(mat.toBreeze() === expected) + } + + test("toRowMatrix") { + val idxRowMat = new IndexedRowMatrix(indexedRows) + val rowMat = idxRowMat.toRowMatrix() + assert(rowMat.numCols() === n) + assert(rowMat.numRows() === 3, "should drop empty rows") + assert(rowMat.rows.collect().toSeq === data.map(_.vector).toSeq) + } + + test("multiply a local matrix") { + val A = new IndexedRowMatrix(indexedRows) + val B = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val C = A.multiply(B) + val localA = A.toBreeze() + val localC = C.toBreeze() + val expected = localA * B.toBreeze.asInstanceOf[BDM[Double]] + assert(localC === expected) + } + + test("gram") { + val A = new IndexedRowMatrix(indexedRows) + val G = A.computeGramianMatrix() + val expected = BDM( + (90.0, 12.0, 24.0), + (12.0, 17.0, 22.0), + (24.0, 22.0, 30.0)) + assert(G.toBreeze === expected) + } + + test("svd") { + val A = new IndexedRowMatrix(indexedRows) + val svd = A.computeSVD(n, computeU = true) + assert(svd.U.isInstanceOf[IndexedRowMatrix]) + val localA = A.toBreeze() + val U = svd.U.toBreeze() + val s = svd.s.toBreeze.asInstanceOf[BDV[Double]] + val V = svd.V.toBreeze.asInstanceOf[BDM[Double]] + assert(closeToZero(U.t * U - BDM.eye[Double](n))) + assert(closeToZero(V.t * V - BDM.eye[Double](n))) + assert(closeToZero(U * brzDiag(s) * V.t - localA)) + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } +} + 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 6422d4e15565d..410413621c44c 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 @@ -73,6 +73,17 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } + test("toBreeze") { + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (6.0, 7.0, 8.0), + (9.0, 0.0, 1.0)) + for (mat <- Seq(denseMat, sparseMat)) { + assert(mat.toBreeze() === expected) + } + } + test("gram") { val expected = Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) From 0135193dcafacf2012bd6b2b0d8cba1daaa8f2e1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 17:28:19 -0700 Subject: [PATCH 11/12] address Reza's comments --- .../spark/examples/mllib/TallSkinnyPCA.scala | 8 +-- .../spark/examples/mllib/TallSkinnySVD.scala | 8 +-- .../apache/spark/mllib/linalg/Matrices.scala | 10 +++ .../mllib/linalg/distributed/RowMatrix.scala | 4 ++ .../linalg/distributed/RowMatrixSuite.scala | 70 ++++++++++++------- 5 files changed, 65 insertions(+), 35 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 4d5560c5350b2..a177435e606ab 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -27,11 +27,11 @@ import org.apache.spark.mllib.linalg.Vectors * The input matrix must be stored in row-oriented dense format, one line per row with its entries * separated by space. For example, * {{{ - * 0 1 - * 2 3 - * 4 5 + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 * }}} - * represents a 3-by-2 matrix, whose first row is (0, 1). + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). */ object TallSkinnyPCA { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 0b92d7c934207..49d09692c8e4a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -27,11 +27,11 @@ import org.apache.spark.mllib.linalg.Vectors * The input matrix must be stored in row-oriented dense format, one line per row with its entries * separated by space. For example, * {{{ - * 0 1 - * 2 3 - * 4 5 + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 * }}} - * represents a 3-by-2 matrix, whose first row is (0, 1). + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). */ object TallSkinnySVD { def main(args: Array[String]) { 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 51b2b0bcfd3c6..b11ba5d30fbd3 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 @@ -38,10 +38,20 @@ trait Matrix extends Serializable { /** Gets the (i, j)-th element. */ private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) + + override def toString: String = toBreeze.toString() } /** * Column-majored dense matrix. + * The entry values are stored in a single array of doubles with columns listed in sequence. + * For example, the following matrix + * {{{ + * 1.0 2.0 + * 3.0 4.0 + * 5.0 6.0 + * }}} + * is stored as `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]`. * * @param numRows number of rows * @param numCols number of columns diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 24f6b72632b2d..f59811f18a68f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -133,6 +133,10 @@ class RowMatrix( } val sk = i + if (sk < k) { + logWarning(s"Requested $k singular values but only found $sk nonzeros.") + } + val s = Vectors.dense(util.Arrays.copyOfRange(sigmas.data, 0, sk)) val V = Matrices.dense(n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) 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 410413621c44c..71ee8e8a4f6fd 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 @@ -19,10 +19,10 @@ package org.apache.spark.mllib.linalg.distributed import org.scalatest.FunSuite -import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, diag => brzDiag, norm => brzNorm} +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector, Matrix} +import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} class RowMatrixSuite extends FunSuite with LocalSparkContext { @@ -42,9 +42,10 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { Vectors.sparse(3, Seq((0, 9.0), (2, 1.0))) ) - val principalComponents = Matrices.dense(n, n, - Array(0.0, math.sqrt(2.0) / 2.0, math.sqrt(2.0) / 2.0, 1.0, 0.0, 0.0, - 0.0, math.sqrt(2.0) / 2.0, - math.sqrt(2.0) / 2.0)) + val principalComponents = BDM( + (0.0, 1.0, 0.0), + (math.sqrt(2.0) / 2.0, 0.0, math.sqrt(2.0) / 2.0), + (math.sqrt(2.0) / 2.0, 0.0, - math.sqrt(2.0) / 2.0)) var denseMat: RowMatrix = _ var sparseMat: RowMatrix = _ @@ -93,24 +94,41 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } - test("svd") { - val A = new BDM[Double](m, n, arr) + test("svd of a full-rank matrix") { for (mat <- Seq(denseMat, sparseMat)) { - val svd = mat.computeSVD(n, computeU = true) - val U = svd.U - val brzSigma = svd.s.toBreeze.asInstanceOf[BDV[Double]] - val brzV = svd.V.toBreeze.asInstanceOf[BDM[Double]] - val rows = U.rows.collect() - val brzUt = new BDM[Double](n, m, rows.flatMap(r => r.toArray)) - val UsVt = brzUt.t * brzDiag(brzSigma) * brzV.t - assert(closeToZero(UsVt - A)) - val VtV: BDM[Double] = brzV.t * brzV - assert(closeToZero(VtV - BDM.eye[Double](n))) - val UtU = U.computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] - assert(closeToZero(UtU - BDM.eye[Double](n))) + val localMat = mat.toBreeze() + val (localU, localSigma, localVt) = brzSvd(localMat) + val localV: BDM[Double] = localVt.t.toDenseMatrix + for (k <- 1 to n) { + val svd = mat.computeSVD(k, computeU = true) + val U = svd.U + val s = svd.s + val V = svd.V + assert(U.numRows() === m) + assert(U.numCols() === k) + assert(s.size === k) + assert(V.numRows === n) + assert(V.numCols === k) + assertColumnEqualUpToSign(U.toBreeze(), localU, k) + assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) + assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + } + val svdWithoutU = mat.computeSVD(n) + assert(svdWithoutU.U === null) } } + test("svd of a low-rank matrix") { + val rows = sc.parallelize(Array.fill(4)(Vectors.dense(1.0, 1.0)), 2) + val mat = new RowMatrix(rows, 4, 2) + val svd = mat.computeSVD(2, computeU = true) + assert(svd.s.size === 1, "should not return zero singular values") + assert(svd.U.numRows() === 4) + assert(svd.U.numCols() === 1) + assert(svd.V.numRows === 2) + assert(svd.V.numCols === 1) + } + def closeToZero(G: BDM[Double]): Boolean = { G.valuesIterator.map(math.abs).sum < 1e-6 } @@ -119,15 +137,13 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { brzNorm(v, 1.0) < 1e-6 } - def assertPrincipalComponentsEqual(a: Matrix, b: Matrix, k: Int) { - val brzA = a.toBreeze.asInstanceOf[BDM[Double]] - val brzB = b.toBreeze.asInstanceOf[BDM[Double]] - assert(brzA.rows === brzB.rows) + def assertColumnEqualUpToSign(A: BDM[Double], B: BDM[Double], k: Int) { + assert(A.rows === B.rows) for (j <- 0 until k) { - val aj = brzA(::, j) - val bj = brzB(::, j) + val aj = A(::, j) + val bj = B(::, j) assert(closeToZero(aj - bj) || closeToZero(aj + bj), - s"The $j-th components mismatch: $aj and $bj") + s"The $j-th columns mismatch: $aj and $bj") } } @@ -136,7 +152,7 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { val pc = denseMat.computePrincipalComponents(k) assert(pc.numRows === n) assert(pc.numCols === k) - assertPrincipalComponentsEqual(pc, principalComponents, k) + assertColumnEqualUpToSign(pc.toBreeze.asInstanceOf[BDM[Double]], principalComponents, k) } } From 24d829460e3ddcfe6c5515d7c197477356c5df0f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 22:06:32 -0700 Subject: [PATCH 12/12] fix for groupBy returning Iterable --- .../spark/mllib/linalg/distributed/CoordinateMatrix.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 710dbbdae3659..9194f657494b2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -75,7 +75,7 @@ class CoordinateMatrix( val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) .groupByKey() .map { case (i, vectorEntries) => - IndexedRow(i, Vectors.sparse(n, vectorEntries)) + IndexedRow(i, Vectors.sparse(n, vectorEntries.toSeq)) } new IndexedRowMatrix(indexedRows, numRows(), n) }