More bug fixes

This commit is contained in:
Reynold Xin 2013-06-30 23:07:40 -07:00
parent c90967a6a2
commit 0791581346
5 changed files with 52 additions and 21 deletions

View file

@ -51,7 +51,7 @@ object Analytics extends Logging {
(me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather
(a: Double, b: Double) => a + b, // merge
1.0,
numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) }
numIter).mapVertices{ case Vertex(id, (outDeg, r)) => r }
}
// /**

View file

@ -252,6 +252,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] {
*
* @tparam U the type of entry in the table of updates
* @tparam VD2 the new vertex value type
*
* @param table the table to join with the vertices in the graph. The table
* should contain at most one entry for each vertex.
* @param mapFunc the function used to compute the new vertex values. The
@ -329,24 +330,26 @@ object Graph {
import spark.graph.impl._
import spark.SparkContext._
def apply(rawEdges: RDD[(Int,Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = {
// Reduce to unique edges
val edges =
if(uniqueEdges) rawEdges.map{ case (s,t) => ((s,t),1) }.reduceByKey( _ + _ )
.map{ case ((s,t), cnt) => Edge(s,t,cnt) }
else rawEdges.map{ case (s,t) => Edge(s,t,1) }
def apply(rawEdges: RDD[(Int, Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = {
// Reduce to unique edges.
val edges: RDD[Edge[Int]] =
if (uniqueEdges) {
rawEdges.map((_, 1)).reduceByKey(_ + _).map { case ((s, t), cnt) => Edge(s, t, cnt) }
} else {
rawEdges.map { case (s, t) => Edge(s, t, 1) }
}
// Determine unique vertices
val vertices = edges.flatMap{ case Edge(s, t, cnt) => Array((s,1), (t,1)) }.reduceByKey( _ + _ )
val vertices: RDD[Vertex[Int]] = edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) }
.reduceByKey(_ + _)
.map{ case (id, deg) => Vertex(id, deg) }
// Return graph
new GraphImpl(vertices, edges)
}
def apply[VD: ClassManifest, ED: ClassManifest](vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = {
def apply[VD: ClassManifest, ED: ClassManifest](
vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = {
new GraphImpl(vertices, edges)
}
implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops
}

View file

@ -1,6 +1,6 @@
package spark.graph.impl
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.ArrayBuilder
import it.unimi.dsi.fastutil.ints.IntArrayList
@ -13,21 +13,25 @@ import spark.graph._
private[graph]
class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] {
private var _data: Array[ED] = _
private var _dataBuilder = ArrayBuilder.make[ED]
val srcIds: IntArrayList = new IntArrayList
val dstIds: IntArrayList = new IntArrayList
// TODO: Specialize data.
val data: ArrayBuffer[ED] = new ArrayBuffer[ED]
def data: Array[ED] = _data
/** Add a new edge to the partition. */
def add(src: Vid, dst: Vid, d: ED) {
srcIds.add(src)
dstIds.add(dst)
data += d
_dataBuilder += d
}
def trim() {
srcIds.trim()
dstIds.trim()
_data = _dataBuilder.result()
}
def size: Int = srcIds.size
@ -41,7 +45,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
override def next(): Edge[ED] = {
edge.src = srcIds.get(pos)
edge.dst = dstIds.get(pos)
edge.data = data(pos)
edge.data = _data(pos)
pos += 1
edge
}

View file

@ -150,7 +150,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected (
}
/**
* Same as mapReduceNeighborhood but map function can return none and there is no default value.
* Same as aggregateNeighbors but map function can return none and there is no default value.
* As a consequence, the resulting table may be much smaller than the set of vertices.
*/
override def aggregateNeighbors[VD2: ClassManifest](
@ -165,7 +165,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected (
part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) }
}, preservesPartitioning = true)
(new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable))
new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)
.mapPartitions { part =>
val (vmap, edges) = part.next()
val edgeSansAcc = new EdgeTriplet[VD, ED]()
@ -188,7 +188,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected (
}
if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) {
e.dst.data._2 =
if (e.dst.data._2.isEmpty) {
if (e.src.data._2.isEmpty) {
mapFunc(edgeSansAcc.src.id, edgeSansAcc)
} else {
val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc)
@ -218,7 +218,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected (
}
}, preservesPartitioning = true).cache()
new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable)
new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable)
}
override def joinVertices[U: ClassManifest](
@ -239,7 +239,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected (
}
}, preservesPartitioning = true).cache()
new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable)
new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable)
}
@ -307,6 +307,7 @@ object GraphImpl {
.mapPartitionsWithIndex({ (pid, iter) =>
val edgePartition = new EdgePartition[ED]
iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) }
edgePartition.trim()
Iterator((pid, edgePartition))
}, preservesPartitioning = true)
}

View file

@ -3,10 +3,33 @@ package spark.graph
import org.scalatest.FunSuite
import spark.SparkContext
import spark.graph.impl.GraphImpl
class GraphSuite extends FunSuite with LocalSparkContext {
test("aggregateNeighbors") {
}
test("joinVertices") {
sc = new SparkContext("local", "test")
val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"), Vertex(3, "three")), 2)
val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo"))))
val g: Graph[String, String] = new GraphImpl(vertices, edges)
val tbl = sc.parallelize(Seq((1, 10), (2, 20)))
val g1 = g.joinVertices(tbl, (v: Vertex[String], u: Int) => v.data + u)
val v = g1.vertices.collect().sortBy(_.id)
assert(v(0).data === "one10")
assert(v(1).data === "two20")
assert(v(2).data === "three")
val e = g1.edges.collect()
assert(e(0).data === "onetwo")
}
// test("graph partitioner") {
// sc = new SparkContext("local", "test")
// val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two")))