Skip to content

Commit

Permalink
Revert
Browse files Browse the repository at this point in the history
  • Loading branch information
holdenk committed Apr 8, 2014
1 parent 7e533f7 commit b692868
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 27 deletions.
14 changes: 0 additions & 14 deletions mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,6 @@ class SVD {
val m = matrix.m
val n = matrix.n

println("matrix "+matrix)
if (m < n || m <= 0 || n <= 0) {
throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n")
}
Expand All @@ -130,7 +129,6 @@ class SVD {
val rowIndices = matrix.rows.map(_.i)

// compute SVD
println("Computing SVD")
val (u, sigma, v) = denseSVD(matrix.rows.map(_.data))

if (computeU) {
Expand Down Expand Up @@ -288,8 +286,6 @@ 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()
println("rows "+rows)
rows.foreach{e => println("r "+e)}
val emits = rows.flatMap {
case (rowind, colsI) =>
val cols = colsI.toList
Expand All @@ -301,26 +297,18 @@ class SVD {
}
}
}.reduceByKey(_ + _)
println("emits ")
emits.foreach{e => println("e "+e)}

// 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)
}
println("ata "+ata)

// Since A^T A is small, we can compute its SVD directly
val svd = Singular.sparseSVD(ata)
println("init svd")
svd.foreach{x => println(x)}
println("done")
val V = svd(0)
println("V" + V)
// This will be updated to rcond
val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9)
println("Sigmas "+sigmas)

if (sigmas.size < k) {
throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size)
Expand Down Expand Up @@ -354,8 +342,6 @@ class SVD {
// Multiply A by VS^-1
val aCols = data.map(entry => (entry.j, (entry.i, entry.mval)))
val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2)))
aCols.foreach{x => println("a "+x)}
bRows.foreach{x => println("b "+x)}
val retUdata = aCols.join(bRows).map {
case (key, ((rowInd, rowVal), (colInd, colVal))) =>
((rowInd, colInd), rowVal * colVal)
Expand Down
25 changes: 12 additions & 13 deletions mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -87,17 +87,17 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll {
val retu = getDenseMatrix(u)
val rets = getDenseMatrix(s)
val retv = getDenseMatrix(v)


// check individual decomposition
// 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)
assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA)
}
/*

test("dense full rank matrix svd") {
val m = 10
val n = 3
Expand All @@ -120,7 +120,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll {
val retv = new DoubleMatrix(v)


// check individual decomposition
// check individual decomposition
assertMatrixApproximatelyEquals(retu, svd(0))
assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1)))
assertMatrixApproximatelyEquals(retv, svd(2))
Expand All @@ -131,7 +131,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll {

test("rank one matrix svd") {
val m = 10
val n = 3
val n = 3
val data = sc.makeRDD(Array.tabulate(m, n){ (a,b) =>
MatrixEntry(a, b, 1.0) }.flatten )
val k = 1
Expand All @@ -153,13 +153,13 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll {
val rets = getDenseMatrix(s)
val retv = getDenseMatrix(v)

// check individual decomposition
// 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)
assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA)
}

test("truncated with k") {
Expand All @@ -168,7 +168,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll {
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)
Expand All @@ -185,11 +185,10 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll {
val retv = getDenseMatrix(v)

assert(retrank == 1, "rank returned not one")
// check individual decomposition
// check individual decomposition
assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0))
assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0)))
assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0))
}
*/
}

0 comments on commit b692868

Please sign in to comment.