Skip to content

Commit

Permalink
Merge pull request #6 from markhamstra/master-csd
Browse files Browse the repository at this point in the history
Merge 0.8.0-candidate-csd branch to master-csd
  • Loading branch information
vnivargi committed Jun 27, 2013
2 parents 04d0e53 + 211a544 commit 928aa6f
Show file tree
Hide file tree
Showing 401 changed files with 20,490 additions and 6,424 deletions.
3 changes: 3 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -34,3 +34,6 @@ log/
spark-tests.log
streaming-tests.log
dependency-reduced-pom.xml
.ensime
.ensime_lucene
derby.log
9 changes: 7 additions & 2 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -12,11 +12,16 @@ This README file only contains basic setup instructions.

## Building

Spark requires Scala 2.9.2. The project is built using Simple Build Tool (SBT),
which is packaged with it. To build Spark and its example programs, run:
Spark requires Scala 2.9.2 (Scala 2.10 is not yet supported). The project is
built using Simple Build Tool (SBT), which is packaged with it. To build
Spark and its example programs, run:

sbt/sbt package

Spark also supports building using Maven. If you would like to build using Maven,
see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html)
in the spark documentation..

To run Spark, you will need to have Scala's bin directory 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
Expand Down
39 changes: 38 additions & 1 deletion bagel/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@
<parent>
<groupId>org.spark-project</groupId>
<artifactId>spark-parent</artifactId>
<version>0.7.0-csd-4-SNAPSHOT</version>
<version>0.8.0-candidate-csd-7-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

Expand Down Expand Up @@ -104,5 +104,42 @@
</plugins>
</build>
</profile>
<profile>
<id>hadoop2-yarn</id>
<dependencies>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-core</artifactId>
<version>${project.version}</version>
<classifier>hadoop2-yarn</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-api</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-common</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<classifier>hadoop2-yarn</classifier>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>
131 changes: 104 additions & 27 deletions bagel/src/main/scala/spark/bagel/Bagel.scala
Original file line number Diff line number Diff line change
Expand Up @@ -4,21 +4,51 @@ import spark._
import spark.SparkContext._

import scala.collection.mutable.ArrayBuffer
import storage.StorageLevel

object Bagel extends Logging {
def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
C : Manifest, A : Manifest](
val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK

/**
* Runs a Bagel program.
* @param sc [[spark.SparkContext]] to use for the program.
* @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the Key will be
* the vertex id.
* @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often this will be an
* empty array, i.e. sc.parallelize(Array[K, Message]()).
* @param combiner [[spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one
* message before sending (which often involves network I/O).
* @param aggregator [[spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep,
* and provides the result to each vertex in the next superstep.
* @param partitioner [[spark.Partitioner]] partitions values by key
* @param numPartitions number of partitions across which to split the graph.
* Default is the default parallelism of the SparkContext
* @param storageLevel [[spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep.
* Defaults to caching in memory.
* @param compute function that takes a Vertex, optional set of (possibly combined) messages to the Vertex,
* optional Aggregator and the current superstep,
* and returns a set of (Vertex, outgoing Messages) pairs
* @tparam K key
* @tparam V vertex type
* @tparam M message type
* @tparam C combiner
* @tparam A aggregator
* @return an RDD of (K, V) pairs representing the graph after completion of the program
*/
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
C: Manifest, A: Manifest](
sc: SparkContext,
vertices: RDD[(K, V)],
messages: RDD[(K, M)],
combiner: Combiner[M, C],
aggregator: Option[Aggregator[V, A]],
partitioner: Partitioner,
numSplits: Int
numPartitions: Int,
storageLevel: StorageLevel = DEFAULT_STORAGE_LEVEL
)(
compute: (V, Option[C], Option[A], Int) => (V, Array[M])
): RDD[(K, V)] = {
val splits = if (numSplits != 0) numSplits else sc.defaultParallelism
val splits = if (numPartitions != 0) numPartitions else sc.defaultParallelism

var superstep = 0
var verts = vertices
Expand All @@ -32,8 +62,9 @@ object Bagel extends Logging {
val combinedMsgs = msgs.combineByKey(
combiner.createCombiner _, combiner.mergeMsg _, combiner.mergeCombiners _, partitioner)
val grouped = combinedMsgs.groupWith(verts)
val superstep_ = superstep // Create a read-only copy of superstep for capture in closure
val (processed, numMsgs, numActiveVerts) =
comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep))
comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep_), storageLevel)

val timeTaken = System.currentTimeMillis - startTime
logInfo("Superstep %d took %d s".format(superstep, timeTaken / 1000))
Expand All @@ -50,57 +81,103 @@ object Bagel extends Logging {
verts
}

def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
C : Manifest](
/** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default storage level */
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
sc: SparkContext,
vertices: RDD[(K, V)],
messages: RDD[(K, M)],
combiner: Combiner[M, C],
partitioner: Partitioner,
numSplits: Int
numPartitions: Int
)(
compute: (V, Option[C], Int) => (V, Array[M])
): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)

/** Runs a Bagel program with no [[spark.bagel.Aggregator]] */
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
sc: SparkContext,
vertices: RDD[(K, V)],
messages: RDD[(K, M)],
combiner: Combiner[M, C],
partitioner: Partitioner,
numPartitions: Int,
storageLevel: StorageLevel
)(
compute: (V, Option[C], Int) => (V, Array[M])
): RDD[(K, V)] = {
run[K, V, M, C, Nothing](
sc, vertices, messages, combiner, None, partitioner, numSplits)(
sc, vertices, messages, combiner, None, partitioner, numPartitions, storageLevel)(
addAggregatorArg[K, V, M, C](compute))
}

def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
C : Manifest](
/**
* Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]]
* and default storage level
*/
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
sc: SparkContext,
vertices: RDD[(K, V)],
messages: RDD[(K, M)],
combiner: Combiner[M, C],
numPartitions: Int
)(
compute: (V, Option[C], Int) => (V, Array[M])
): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)

/** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default [[spark.HashPartitioner]]*/
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
sc: SparkContext,
vertices: RDD[(K, V)],
messages: RDD[(K, M)],
combiner: Combiner[M, C],
numSplits: Int
numPartitions: Int,
storageLevel: StorageLevel
)(
compute: (V, Option[C], Int) => (V, Array[M])
): RDD[(K, V)] = {
val part = new HashPartitioner(numSplits)
val part = new HashPartitioner(numPartitions)
run[K, V, M, C, Nothing](
sc, vertices, messages, combiner, None, part, numSplits)(
sc, vertices, messages, combiner, None, part, numPartitions, storageLevel)(
addAggregatorArg[K, V, M, C](compute))
}

def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
/**
* Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]],
* [[spark.bagel.DefaultCombiner]] and the default storage level
*/
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
sc: SparkContext,
vertices: RDD[(K, V)],
messages: RDD[(K, M)],
numSplits: Int
numPartitions: Int
)(
compute: (V, Option[Array[M]], Int) => (V, Array[M])
): RDD[(K, V)] = {
val part = new HashPartitioner(numSplits)
): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)

/**
* Runs a Bagel program with no [[spark.bagel.Aggregator]], the default [[spark.HashPartitioner]]
* and [[spark.bagel.DefaultCombiner]]
*/
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
sc: SparkContext,
vertices: RDD[(K, V)],
messages: RDD[(K, M)],
numPartitions: Int,
storageLevel: StorageLevel
)(
compute: (V, Option[Array[M]], Int) => (V, Array[M])
): RDD[(K, V)] = {
val part = new HashPartitioner(numPartitions)
run[K, V, M, Array[M], Nothing](
sc, vertices, messages, new DefaultCombiner(), None, part, numSplits)(
sc, vertices, messages, new DefaultCombiner(), None, part, numPartitions, storageLevel)(
addAggregatorArg[K, V, M, Array[M]](compute))
}

/**
* Aggregates the given vertices using the given aggregator, if it
* is specified.
*/
private def agg[K, V <: Vertex, A : Manifest](
private def agg[K, V <: Vertex, A: Manifest](
verts: RDD[(K, V)],
aggregator: Option[Aggregator[V, A]]
): Option[A] = aggregator match {
Expand All @@ -116,10 +193,11 @@ object Bagel extends Logging {
* function. Returns the processed RDD, the number of messages
* created, and the number of active vertices.
*/
private def comp[K : Manifest, V <: Vertex, M <: Message[K], C](
private def comp[K: Manifest, V <: Vertex, M <: Message[K], C](
sc: SparkContext,
grouped: RDD[(K, (Seq[C], Seq[V]))],
compute: (V, Option[C]) => (V, Array[M])
compute: (V, Option[C]) => (V, Array[M]),
storageLevel: StorageLevel
): (RDD[(K, (V, Array[M]))], Int, Int) = {
var numMsgs = sc.accumulator(0)
var numActiveVerts = sc.accumulator(0)
Expand All @@ -137,7 +215,7 @@ object Bagel extends Logging {
numActiveVerts += 1

Some((newVert, newMsgs))
}.cache
}.persist(storageLevel)

// Force evaluation of processed RDD for accurate performance measurements
processed.foreach(x => {})
Expand All @@ -149,9 +227,7 @@ object Bagel extends Logging {
* Converts a compute function that doesn't take an aggregator to
* one that does, so it can be passed to Bagel.run.
*/
private def addAggregatorArg[
K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C
](
private def addAggregatorArg[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C](
compute: (V, Option[C], Int) => (V, Array[M])
): (V, Option[C], Option[Nothing], Int) => (V, Array[M]) = {
(vert: V, msgs: Option[C], aggregated: Option[Nothing], superstep: Int) =>
Expand All @@ -170,7 +246,8 @@ trait Aggregator[V, A] {
def mergeAggregators(a: A, b: A): A
}

class DefaultCombiner[M : Manifest] extends Combiner[M, Array[M]] with Serializable {
/** Default combiner that simply appends messages together (i.e. performs no aggregation) */
class DefaultCombiner[M: Manifest] extends Combiner[M, Array[M]] with Serializable {
def createCombiner(msg: M): Array[M] =
Array(msg)
def mergeMsg(combiner: Array[M], msg: M): Array[M] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ import scala.xml.{XML,NodeSeq}
object WikipediaPageRank {
def main(args: Array[String]) {
if (args.length < 5) {
System.err.println("Usage: WikipediaPageRank <inputFile> <threshold> <numSplits> <host> <usePartitioner>")
System.err.println("Usage: WikipediaPageRank <inputFile> <threshold> <numPartitions> <host> <usePartitioner>")
System.exit(-1)
}

Expand All @@ -25,7 +25,7 @@ object WikipediaPageRank {

val inputFile = args(0)
val threshold = args(1).toDouble
val numSplits = args(2).toInt
val numPartitions = args(2).toInt
val host = args(3)
val usePartitioner = args(4).toBoolean
val sc = new SparkContext(host, "WikipediaPageRank")
Expand Down Expand Up @@ -69,7 +69,7 @@ object WikipediaPageRank {
val result =
Bagel.run(
sc, vertices, messages, combiner = new PRCombiner(),
numSplits = numSplits)(
numPartitions = numPartitions)(
utils.computeWithCombiner(numVertices, epsilon))

// Print the result
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ object WikipediaPageRankStandalone {
n: Long,
partitioner: Partitioner,
usePartitioner: Boolean,
numSplits: Int
numPartitions: Int
): RDD[(String, Double)] = {
var ranks = links.mapValues { edges => defaultRank }
for (i <- 1 to numIterations) {
Expand Down
Loading

0 comments on commit 928aa6f

Please sign in to comment.