Skip to content
Snippets Groups Projects
Commit df29d0ea authored by Matei Zaharia's avatar Matei Zaharia
Browse files

Initial commit

parents
No related branches found
No related tags found
No related merge requests found
Showing with 718 additions and 0 deletions
*~
*.swp
build
work
Makefile 0 → 100644
EMPTY =
SPACE = $(EMPTY) $(EMPTY)
# Build up classpath by concatenating some strings
JARS = third_party/nexus.jar
JARS += third_party/asm-3.2/lib/all/asm-all-3.2.jar
JARS += third_party/colt.jar
JARS += third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar
JARS += third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar
JARS += third_party/hadoop-0.20.0/lib/commons-logging-1.0.4.jar
JARS += third_party/scalatest-1.0/scalatest-1.0.jar
JARS += third_party/ScalaCheck-1.5.jar
CLASSPATH = $(subst $(SPACE),:,$(JARS))
SCALA_SOURCES = src/examples/*.scala src/scala/spark/*.scala src/scala/spark/repl/*.scala
SCALA_SOURCES += src/test/spark/*.scala src/test/spark/repl/*.scala
JAVA_SOURCES = $(wildcard src/java/spark/compress/lzf/*.java)
ifeq ($(USE_FSC),1)
COMPILER_NAME = fsc
else
COMPILER_NAME = scalac
endif
ifeq ($(SCALA_HOME),)
COMPILER = $(COMPILER_NAME)
else
COMPILER = $(SCALA_HOME)/bin/$(COMPILER_NAME)
endif
all: scala java
build/classes:
mkdir -p build/classes
scala: build/classes java
$(COMPILER) -unchecked -d build/classes -classpath $(CLASSPATH) $(SCALA_SOURCES)
java: $(JAVA_SOURCES) build/classes
javac -d build/classes $(JAVA_SOURCES)
native: java
$(MAKE) -C src/native
jar: build/spark.jar build/spark-dep.jar
build/spark.jar: scala java
jar cf build/spark.jar -C build/classes spark
build/spark-dep.jar:
mkdir -p build/dep
cd build/dep && for i in $(JARS); do jar xf ../../$$i; done
jar cf build/spark-dep.jar -C build/dep .
test: all
./alltests
default: all
clean:
$(MAKE) -C src/native clean
rm -rf build
.phony: default all clean scala java native jar
README 0 → 100644
Spark requires Scala 2.7.7. It will currently not work with 2.8, or with
earlier versions of the 2.7 branch.
To build and run Spark, you will need to have Scala's bin in your $PATH,
or you will need to set the SCALA_HOME environment variable to point
to where you've installed Scala. Scala must be accessible through one
of these methods on Nexus slave nodes as well as on the master.
To build Spark and the example programs, run make.
To run one of the examples, use ./run <class> <params>. For example,
./run SparkLR will run the Logistic Regression example. Each of the
example programs prints usage help if no params are given.
Tip: If you are building Spark and examples repeatedly, export USE_FSC=1
to have the Makefile use the fsc compiler daemon instead of scalac.
#!/bin/bash
FWDIR=`dirname $0`
$FWDIR/run org.scalatest.tools.Runner -p $FWDIR/build/classes -o $@
This diff is collapsed.
run 0 → 100755
#!/bin/bash
# Figure out where the Scala framework is installed
FWDIR=`dirname $0`
# Set JAVA_OPTS to be able to load libnexus.so and set various other misc options
JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx750m"
if [ -e $FWDIR/conf/java-opts ] ; then
JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`"
fi
export JAVA_OPTS
# Build up classpath
CLASSPATH=$FWDIR/build/classes
CLASSPATH+=:$FWDIR/third_party/nexus.jar
CLASSPATH+=:$FWDIR/third_party/asm-3.2/lib/all/asm-all-3.2.jar
CLASSPATH+=:$FWDIR/third_party/colt.jar
CLASSPATH+=:$FWDIR/third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar
CLASSPATH+=:$FWDIR/third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar
CLASSPATH+=:third_party/scalatest-1.0/scalatest-1.0.jar
CLASSPATH+=:third_party/ScalaCheck-1.5.jar
for jar in $FWDIR/third_party/hadoop-0.20.0/lib/*.jar; do
CLASSPATH+=:$jar
done
export CLASSPATH
if [ -n "$SCALA_HOME" ]; then
SCALA=${SCALA_HOME}/bin/scala
else
SCALA=scala
fi
exec $SCALA -cp $CLASSPATH $@
#!/bin/sh
echo "In spark-executor"
FWDIR="`dirname $0`"
echo Framework dir: $FWDIR
exec $FWDIR/run spark.Executor
#!/bin/sh
FWDIR="`dirname $0`"
exec $FWDIR/run spark.repl.Main $@
import spark._
object CpuHog {
def main(args: Array[String]) {
if (args.length != 3) {
System.err.println("Usage: CpuHog <master> <tasks> <threads_per_task>");
System.exit(1)
}
val sc = new SparkContext(args(0), "CPU hog")
val tasks = args(1).toInt
val threads = args(2).toInt
def task {
for (i <- 0 until threads-1) {
new Thread() {
override def run {
while(true) {}
}
}.start()
}
while(true) {}
}
sc.runTasks(Array.make(tasks, () => task))
}
}
import spark._
object HdfsTest {
def main(args: Array[String]) {
val sc = new SparkContext(args(0), "HdfsTest")
val file = sc.textFile(args(1))
val mapped = file.map(s => s.length).cache()
for (iter <- 1 to 10) {
val start = System.currentTimeMillis()
for (x <- mapped) { x + 2 }
// println("Processing: " + x)
val end = System.currentTimeMillis()
println("Iteration " + iter + " took " + (end-start) + " ms")
}
}
}
import java.util.Random
import cern.jet.math._
import cern.colt.matrix._
import cern.colt.matrix.linalg._
object LocalALS {
// Parameters set through command line arguments
var M = 0 // Number of movies
var U = 0 // Number of users
var F = 0 // Number of features
var ITERATIONS = 0
val LAMBDA = 0.01 // Regularization coefficient
// Some COLT objects
val factory2D = DoubleFactory2D.dense
val factory1D = DoubleFactory1D.dense
val algebra = Algebra.DEFAULT
val blas = SeqBlas.seqBlas
def generateR(): DoubleMatrix2D = {
val mh = factory2D.random(M, F)
val uh = factory2D.random(U, F)
return algebra.mult(mh, algebra.transpose(uh))
}
def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D],
us: Array[DoubleMatrix1D]): Double =
{
val r = factory2D.make(M, U)
for (i <- 0 until M; j <- 0 until U) {
r.set(i, j, blas.ddot(ms(i), us(j)))
}
//println("R: " + r)
blas.daxpy(-1, targetR, r)
val sumSqs = r.aggregate(Functions.plus, Functions.square)
return Math.sqrt(sumSqs / (M * U))
}
def updateMovie(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
R: DoubleMatrix2D) : DoubleMatrix1D =
{
val XtX = factory2D.make(F, F)
val Xty = factory1D.make(F)
// For each user that rated the movie
for (j <- 0 until U) {
val u = us(j)
// Add u * u^t to XtX
blas.dger(1, u, u, XtX)
// Add u * rating to Xty
blas.daxpy(R.get(i, j), u, Xty)
}
// Add regularization coefs to diagonal terms
for (d <- 0 until F) {
XtX.set(d, d, XtX.get(d, d) + LAMBDA * U)
}
// Solve it with Cholesky
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
return solved2D.viewColumn(0)
}
def updateUser(j: Int, u: DoubleMatrix1D, ms: Array[DoubleMatrix1D],
R: DoubleMatrix2D) : DoubleMatrix1D =
{
val XtX = factory2D.make(F, F)
val Xty = factory1D.make(F)
// For each movie that the user rated
for (i <- 0 until M) {
val m = ms(i)
// Add m * m^t to XtX
blas.dger(1, m, m, XtX)
// Add m * rating to Xty
blas.daxpy(R.get(i, j), m, Xty)
}
// Add regularization coefs to diagonal terms
for (d <- 0 until F) {
XtX.set(d, d, XtX.get(d, d) + LAMBDA * M)
}
// Solve it with Cholesky
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
return solved2D.viewColumn(0)
}
def main(args: Array[String]) {
args match {
case Array(m, u, f, iters) => {
M = m.toInt
U = u.toInt
F = f.toInt
ITERATIONS = iters.toInt
}
case _ => {
System.err.println("Usage: LocalALS <M> <U> <F> <iters>")
System.exit(1)
}
}
printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS);
val R = generateR()
// Initialize m and u randomly
var ms = Array.fromFunction(_ => factory1D.random(F))(M)
var us = Array.fromFunction(_ => factory1D.random(F))(U)
// Iteratively update movies then users
for (iter <- 1 to ITERATIONS) {
println("Iteration " + iter + ":")
ms = (0 until M).map(i => updateMovie(i, ms(i), us, R)).toArray
us = (0 until U).map(j => updateUser(j, us(j), ms, R)).toArray
println("RMSE = " + rmse(R, ms, us))
println()
}
}
}
import java.util.Random
import Vector._
object LocalFileLR {
val D = 10 // Numer of dimensions
val rand = new Random(42)
case class DataPoint(x: Vector, y: Double)
def parsePoint(line: String): DataPoint = {
val nums = line.split(' ').map(_.toDouble)
return DataPoint(new Vector(nums.subArray(1, D+1)), nums(0))
}
def main(args: Array[String]) {
val lines = scala.io.Source.fromFile(args(0)).getLines
val points = lines.map(parsePoint _)
val ITERATIONS = args(1).toInt
// Initialize w to a random value
var w = Vector(D, _ => 2 * rand.nextDouble - 1)
println("Initial w: " + w)
for (i <- 1 to ITERATIONS) {
println("On iteration " + i)
var gradient = Vector.zeros(D)
for (p <- points) {
val scale = (1 / (1 + Math.exp(-p.y * (w dot p.x))) - 1) * p.y
gradient += scale * p.x
}
w -= gradient
}
println("Final w: " + w)
}
}
import java.util.Random
import Vector._
object LocalLR {
val N = 10000 // Number of data points
val D = 10 // Numer of dimensions
val R = 0.7 // Scaling factor
val ITERATIONS = 5
val rand = new Random(42)
case class DataPoint(x: Vector, y: Double)
def generateData = {
def generatePoint(i: Int) = {
val y = if(i % 2 == 0) -1 else 1
val x = Vector(D, _ => rand.nextGaussian + y * R)
DataPoint(x, y)
}
Array.fromFunction(generatePoint _)(N)
}
def main(args: Array[String]) {
val data = generateData
// Initialize w to a random value
var w = Vector(D, _ => 2 * rand.nextDouble - 1)
println("Initial w: " + w)
for (i <- 1 to ITERATIONS) {
println("On iteration " + i)
var gradient = Vector.zeros(D)
for (p <- data) {
val scale = (1 / (1 + Math.exp(-p.y * (w dot p.x))) - 1) * p.y
gradient += scale * p.x
}
w -= gradient
}
println("Final w: " + w)
}
}
import spark._
import SparkContext._
object LocalPi {
def main(args: Array[String]) {
var count = 0
for (i <- 1 to 100000) {
val x = Math.random * 2 - 1
val y = Math.random * 2 - 1
if (x*x + y*y < 1) count += 1
}
println("Pi is roughly " + 4 * count / 100000.0)
}
}
\ No newline at end of file
import spark._
object SleepJob {
def main(args: Array[String]) {
if (args.length != 3) {
System.err.println("Usage: SleepJob <master> <tasks> <task_duration>");
System.exit(1)
}
val sc = new SparkContext(args(0), "Sleep job")
val tasks = args(1).toInt
val duration = args(2).toInt
def task {
val start = System.currentTimeMillis
while (System.currentTimeMillis - start < duration * 1000L)
Thread.sleep(200)
}
sc.runTasks(Array.make(tasks, () => task))
}
}
import java.io.Serializable
import java.util.Random
import cern.jet.math._
import cern.colt.matrix._
import cern.colt.matrix.linalg._
import spark._
object SparkALS {
// Parameters set through command line arguments
var M = 0 // Number of movies
var U = 0 // Number of users
var F = 0 // Number of features
var ITERATIONS = 0
val LAMBDA = 0.01 // Regularization coefficient
// Some COLT objects
val factory2D = DoubleFactory2D.dense
val factory1D = DoubleFactory1D.dense
val algebra = Algebra.DEFAULT
val blas = SeqBlas.seqBlas
def generateR(): DoubleMatrix2D = {
val mh = factory2D.random(M, F)
val uh = factory2D.random(U, F)
return algebra.mult(mh, algebra.transpose(uh))
}
def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D],
us: Array[DoubleMatrix1D]): Double =
{
val r = factory2D.make(M, U)
for (i <- 0 until M; j <- 0 until U) {
r.set(i, j, blas.ddot(ms(i), us(j)))
}
//println("R: " + r)
blas.daxpy(-1, targetR, r)
val sumSqs = r.aggregate(Functions.plus, Functions.square)
return Math.sqrt(sumSqs / (M * U))
}
def updateMovie(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
R: DoubleMatrix2D) : DoubleMatrix1D =
{
val U = us.size
val F = us(0).size
val XtX = factory2D.make(F, F)
val Xty = factory1D.make(F)
// For each user that rated the movie
for (j <- 0 until U) {
val u = us(j)
// Add u * u^t to XtX
blas.dger(1, u, u, XtX)
// Add u * rating to Xty
blas.daxpy(R.get(i, j), u, Xty)
}
// Add regularization coefs to diagonal terms
for (d <- 0 until F) {
XtX.set(d, d, XtX.get(d, d) + LAMBDA * U)
}
// Solve it with Cholesky
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
return solved2D.viewColumn(0)
}
def updateUser(j: Int, u: DoubleMatrix1D, ms: Array[DoubleMatrix1D],
R: DoubleMatrix2D) : DoubleMatrix1D =
{
val M = ms.size
val F = ms(0).size
val XtX = factory2D.make(F, F)
val Xty = factory1D.make(F)
// For each movie that the user rated
for (i <- 0 until M) {
val m = ms(i)
// Add m * m^t to XtX
blas.dger(1, m, m, XtX)
// Add m * rating to Xty
blas.daxpy(R.get(i, j), m, Xty)
}
// Add regularization coefs to diagonal terms
for (d <- 0 until F) {
XtX.set(d, d, XtX.get(d, d) + LAMBDA * M)
}
// Solve it with Cholesky
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
return solved2D.viewColumn(0)
}
def main(args: Array[String]) {
var host = ""
var slices = 0
args match {
case Array(m, u, f, iters, slices_, host_) => {
M = m.toInt
U = u.toInt
F = f.toInt
ITERATIONS = iters.toInt
slices = slices_.toInt
host = host_
}
case _ => {
System.err.println("Usage: SparkALS <M> <U> <F> <iters> <slices> <host>")
System.exit(1)
}
}
printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS);
val spark = new SparkContext(host, "SparkALS")
val R = generateR()
// Initialize m and u randomly
var ms = Array.fromFunction(_ => factory1D.random(F))(M)
var us = Array.fromFunction(_ => factory1D.random(F))(U)
// Iteratively update movies then users
val Rc = spark.broadcast(R)
var msb = spark.broadcast(ms)
var usb = spark.broadcast(us)
for (iter <- 1 to ITERATIONS) {
println("Iteration " + iter + ":")
ms = spark.parallelize(0 until M, slices)
.map(i => updateMovie(i, msb.value(i), usb.value, Rc.value))
.toArray
msb = spark.broadcast(ms) // Re-broadcast ms because it was updated
us = spark.parallelize(0 until U, slices)
.map(i => updateUser(i, usb.value(i), msb.value, Rc.value))
.toArray
usb = spark.broadcast(us) // Re-broadcast us because it was updated
println("RMSE = " + rmse(R, ms, us))
println()
}
}
}
import java.util.Random
import Vector._
import spark._
object SparkHdfsLR {
val D = 10 // Numer of dimensions
val rand = new Random(42)
case class DataPoint(x: Vector, y: Double)
def parsePoint(line: String): DataPoint = {
//val nums = line.split(' ').map(_.toDouble)
//return DataPoint(new Vector(nums.subArray(1, D+1)), nums(0))
val tok = new java.util.StringTokenizer(line, " ")
var y = tok.nextToken.toDouble
var x = new Array[Double](D)
var i = 0
while (i < D) {
x(i) = tok.nextToken.toDouble; i += 1
}
return DataPoint(new Vector(x), y)
}
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println("Usage: SparkHdfsLR <master> <file> <iters>")
System.exit(1)
}
val sc = new SparkContext(args(0), "SparkHdfsLR")
val lines = sc.textFile(args(1))
val points = lines.map(parsePoint _).cache()
val ITERATIONS = args(2).toInt
// Initialize w to a random value
var w = Vector(D, _ => 2 * rand.nextDouble - 1)
println("Initial w: " + w)
for (i <- 1 to ITERATIONS) {
println("On iteration " + i)
val gradient = sc.accumulator(Vector.zeros(D))
for (p <- points) {
val scale = (1 / (1 + Math.exp(-p.y * (w dot p.x))) - 1) * p.y
gradient += scale * p.x
}
w -= gradient.value
}
println("Final w: " + w)
}
}
import java.util.Random
import Vector._
import spark._
object SparkLR {
val N = 10000 // Number of data points
val D = 10 // Numer of dimensions
val R = 0.7 // Scaling factor
val ITERATIONS = 5
val rand = new Random(42)
case class DataPoint(x: Vector, y: Double)
def generateData = {
def generatePoint(i: Int) = {
val y = if(i % 2 == 0) -1 else 1
val x = Vector(D, _ => rand.nextGaussian + y * R)
DataPoint(x, y)
}
Array.fromFunction(generatePoint _)(N)
}
def main(args: Array[String]) {
if (args.length == 0) {
System.err.println("Usage: SparkLR <host> [<slices>]")
System.exit(1)
}
val sc = new SparkContext(args(0), "SparkLR")
val numSlices = if (args.length > 1) args(1).toInt else 2
val data = generateData
// Initialize w to a random value
var w = Vector(D, _ => 2 * rand.nextDouble - 1)
println("Initial w: " + w)
for (i <- 1 to ITERATIONS) {
println("On iteration " + i)
val gradient = sc.accumulator(Vector.zeros(D))
for (p <- sc.parallelize(data, numSlices)) {
val scale = (1 / (1 + Math.exp(-p.y * (w dot p.x))) - 1) * p.y
gradient += scale * p.x
}
w -= gradient.value
}
println("Final w: " + w)
}
}
import spark._
import SparkContext._
object SparkPi {
def main(args: Array[String]) {
if (args.length == 0) {
System.err.println("Usage: SparkLR <host> [<slices>]")
System.exit(1)
}
val spark = new SparkContext(args(0), "SparkPi")
val slices = if (args.length > 1) args(1).toInt else 2
var count = spark.accumulator(0)
for (i <- spark.parallelize(1 to 100000, slices)) {
val x = Math.random * 2 - 1
val y = Math.random * 2 - 1
if (x*x + y*y < 1) count += 1
}
println("Pi is roughly " + 4 * count.value / 100000.0)
}
}
\ No newline at end of file
@serializable class Vector(val elements: Array[Double]) {
def length = elements.length
def apply(index: Int) = elements(index)
def + (other: Vector): Vector = {
if (length != other.length)
throw new IllegalArgumentException("Vectors of different length")
return Vector(length, i => this(i) + other(i))
}
def - (other: Vector): Vector = {
if (length != other.length)
throw new IllegalArgumentException("Vectors of different length")
return Vector(length, i => this(i) - other(i))
}
def dot(other: Vector): Double = {
if (length != other.length)
throw new IllegalArgumentException("Vectors of different length")
var ans = 0.0
for (i <- 0 until length)
ans += this(i) * other(i)
return ans
}
def * ( scale: Double): Vector = Vector(length, i => this(i) * scale)
def unary_- = this * -1
def sum = elements.reduceLeft(_ + _)
override def toString = elements.mkString("(", ", ", ")")
}
object Vector {
def apply(elements: Array[Double]) = new Vector(elements)
def apply(elements: Double*) = new Vector(elements.toArray)
def apply(length: Int, initializer: Int => Double): Vector = {
val elements = new Array[Double](length)
for (i <- 0 until length)
elements(i) = initializer(i)
return new Vector(elements)
}
def zeros(length: Int) = new Vector(new Array[Double](length))
def ones(length: Int) = Vector(length, _ => 1)
class Multiplier(num: Double) {
def * (vec: Vector) = vec * num
}
implicit def doubleToMultiplier(num: Double) = new Multiplier(num)
implicit object VectorAccumParam extends spark.AccumulatorParam[Vector] {
def add(t1: Vector, t2: Vector) = t1 + t2
def zero(initialValue: Vector) = Vector.zeros(initialValue.length)
}
}
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment