aboutsummaryrefslogtreecommitdiff
path: root/mllib
diff options
context:
space:
mode:
authorReza Zadeh <rizlar@gmail.com>2014-01-04 11:30:36 -0800
committerReza Zadeh <rizlar@gmail.com>2014-01-04 11:30:36 -0800
commit35adc72794f25223502562f2dc0077f61d91cb79 (patch)
tree86f078c804115177e20eb60a2fc176e7583bad54 /mllib
parent73daa700bd2acff7ff196c9262dffb2d8b9354bf (diff)
downloadspark-35adc72794f25223502562f2dc0077f61d91cb79.tar.gz
spark-35adc72794f25223502562f2dc0077f61d91cb79.tar.bz2
spark-35adc72794f25223502562f2dc0077f61d91cb79.zip
set methods
Diffstat (limited to 'mllib')
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala59
1 files changed, 52 insertions, 7 deletions
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 465fc746ed..9703e84312 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
@@ -24,6 +24,50 @@ import org.apache.spark.rdd.RDD
import org.jblas.{DoubleMatrix, Singular, MatrixFunctions}
+
+/**
+ * Class used to obtain singular value decompositions
+ * @param data Matrix in sparse matrix format
+ * @param m number of rows
+ * @param n number of columns
+ */
+class GradientDescent(var data: RDD[MatrixEntry], var m: Int, var n: Int) {
+ private var k: Int = 1
+
+ /**
+ * Set the number of top-k singular vectors to return
+ */
+ def setK(k: Int): this.type = {
+ this.k = k
+ this
+ }
+
+ /**
+ * Set matrix to be used for SVD
+ */
+ def setDatadata(data: RDD[MatrixEntry]): this.type = {
+ this.data = data
+ this
+ }
+
+ /**
+ * Set dimensions of matrix: rows
+ */
+ def setNumRows(m: Int): this.type = {
+ this.m = m
+ this
+ }
+
+ /**
+ * Set dimensions of matrix: columns
+ */
+ def setNumCols(n: Int): this.type = {
+ this.n = n
+ this
+ }
+}
+
+
/**
* Top-level methods for calling Singular Value Decomposition
* NOTE: All matrices are in 1-indexed sparse format RDD[((int, int), value)]
@@ -76,7 +120,7 @@ object 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().cache()
+ (entry.i, (entry.j, entry.mval))).groupByKey()
val emits = rows.flatMap{ case (rowind, cols) =>
cols.flatMap{ case (colind1, mval1) =>
cols.map{ case (colind2, mval2) =>
@@ -85,7 +129,7 @@ object SVD {
// Construct jblas A^T A locally
val ata = DoubleMatrix.zeros(n, n)
- for(entry <- emits.toArray) {
+ for (entry <- emits.toArray) {
ata.put(entry._1._1-1, entry._1._2-1, entry._2)
}
@@ -94,7 +138,7 @@ object SVD {
val V = svd(0)
val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9)
- if(sigmas.size < k) {
+ if (sigmas.size < k) {
throw new Exception("Not enough singular values to return")
}
@@ -105,14 +149,15 @@ object SVD {
// prepare V for returning
val retV = sc.makeRDD(
Array.tabulate(V.rows, sigma.length){ (i,j) =>
- MatrixEntry(i+1, j+1, V.get(i,j)) }.flatten)
+ MatrixEntry(i + 1, j + 1, V.get(i,j)) }.flatten)
- val retS = sc.makeRDD(Array.tabulate(sigma.length){x=>MatrixEntry(x+1,x+1,sigma(x))})
+ val retS = sc.makeRDD(Array.tabulate(sigma.length){
+ x => MatrixEntry(x + 1,x + 1, sigma(x))})
// Compute U as U = A V S^-1
- // turn V S^-1 into an RDD as a sparse matrix and cache it
+ // turn V S^-1 into an RDD as a sparse matrix
val vsirdd = sc.makeRDD(Array.tabulate(V.rows, sigma.length)
- { (i,j) => ((i+1, j+1), V.get(i,j)/sigma(j)) }.flatten).cache()
+ { (i,j) => ((i + 1, j + 1), V.get(i,j)/sigma(j)) }.flatten)
// Multiply A by VS^-1
val aCols = data.map(entry => (entry.j, (entry.i, entry.mval)))