diff options
author | Matei Zaharia <matei@databricks.com> | 2014-01-22 14:01:30 -0800 |
---|---|---|
committer | Matei Zaharia <matei@databricks.com> | 2014-01-22 14:01:30 -0800 |
commit | d009b17d137edf2f1b9da04254e55fb7455faa3d (patch) | |
tree | 9265b72663c85f4a25f40edf00154ac7e3536f2f /examples/src | |
parent | 749f842827c7e7766a342b6b0a437803044a9f90 (diff) | |
parent | 85b95d039ddfc7a2b2b27f506852859181ed16c1 (diff) | |
download | spark-d009b17d137edf2f1b9da04254e55fb7455faa3d.tar.gz spark-d009b17d137edf2f1b9da04254e55fb7455faa3d.tar.bz2 spark-d009b17d137edf2f1b9da04254e55fb7455faa3d.zip |
Merge pull request #315 from rezazadeh/sparsesvd
Sparse SVD
# Singular Value Decomposition
Given an *m x n* matrix *A*, compute matrices *U, S, V* such that
*A = U * S * V^T*
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^TA = V S^2 V^T*,
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 singular vectors associated with the largest k singular values
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^T*U = eye(k).
* *V* is *n x k* and satisfies V^TV = 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.
# Testing
Tests included. They test:
- Decomposition promise (A = USV^T)
- For small matrices, output is compared to that of jblas
- Rank 1 matrix test included
- Full Rank matrix test included
- Middle-rank matrix forced via k included
# Example Usage
import org.apache.spark.SparkContext
import org.apache.spark.mllib.linalg.SVD
import org.apache.spark.mllib.linalg.SparseMatrix
import org.apache.spark.mllib.linalg.MatrixyEntry
// Load and parse the data file
val data = sc.textFile("mllib/data/als/test.data").map { line =>
val parts = line.split(',')
MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble)
}
val m = 4
val n = 4
// recover top 1 singular vector
val decomposed = SVD.sparseSVD(SparseMatrix(data, m, n), 1)
println("singular values = " + decomposed.S.data.toArray.mkString)
# Documentation
Added to docs/mllib-guide.md
Diffstat (limited to 'examples/src')
-rw-r--r-- | examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala | 59 |
1 files changed, 59 insertions, 0 deletions
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 new file mode 100644 index 0000000000..19676fcc1a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala @@ -0,0 +1,59 @@ +/* + * 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 +import org.apache.spark.mllib.linalg.MatrixEntry +import org.apache.spark.mllib.linalg.SparseMatrix + +/** + * 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 <master> <file> m n") + System.exit(1) + } + val sc = new SparkContext(args(0), "SVD", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + + // 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 = SVD.sparseSVD(SparseMatrix(data, m, n), 1) + val u = decomposed.U.data + val s = decomposed.S.data + val v = decomposed.V.data + + println("singular values = " + s.toArray.mkString) + } +} |