aboutsummaryrefslogtreecommitdiff
path: root/mllib
diff options
context:
space:
mode:
authorReza Zadeh <rizlar@gmail.com>2013-12-26 18:09:33 -0500
committerReza Zadeh <rizlar@gmail.com>2013-12-26 18:09:33 -0500
commit1a21ba29672074692be6c006a8938bfa86330a19 (patch)
treea736e506a1510992950aa461ab4b26920f839cab /mllib
parent6c3674cd235558ec09e6b97382bb541b379a3f8f (diff)
downloadspark-1a21ba29672074692be6c006a8938bfa86330a19.tar.gz
spark-1a21ba29672074692be6c006a8938bfa86330a19.tar.bz2
spark-1a21ba29672074692be6c006a8938bfa86330a19.zip
new main file
Diffstat (limited to 'mllib')
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/sparsesvd.scala29
1 files changed, 19 insertions, 10 deletions
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/sparsesvd.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/sparsesvd.scala
index f9b9a04f19..be8ccff9cc 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/sparsesvd.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/sparsesvd.scala
@@ -110,22 +110,31 @@ object SVD {
(retU, retS, retV)
}
+
def main(args: Array[String]) {
- if (args.length < 4) {
- println("Usage: KMeans <master> <input_file> <k> <max_iterations> [<runs>]")
+ if (args.length < 8) {
+ println("Usage: SVD <master> <matrix_file> <m> <n> <minimum_singular_value> <output_U_file> <output_S_file> <output_V_file>")
System.exit(1)
}
- val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt)
- val runs = if (args.length >= 5) args(4).toInt else 1
- val sc = new SparkContext(master, "KMeans")
- val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)).cache()
- println("Cost: ")
+ val (master, inputFile, m, n, min_svalue, output_u, output_s, output_v) =
+ (args(0), args(1), args(2).toInt, args(3).toInt, args(4).toDouble, 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(',')
+ ((parts(0).toInt, parts(1).toInt), parts(2).toDouble)
+ }
+
+ val (u, s, v) = SVD.sparseSVD(data, m, n, min_svalue)
+ println("Computed " + s.size + " singular values and vectors")
+ u.saveAsText(output_u)
+ s.saveAsText(output_s)
+ v.saveAsText(output_v)
System.exit(0)
- //val data = sc.makeRDD(Array.tabulate(m,n){ (a,b)=> ((a+1,b+1),a*b%37) }.flatten )
}
}
-
-