forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request apache#3 from dcrankshaw/osdi_with_kcore_for_merge
Osdi with kcore for merge
- Loading branch information
Showing
5 changed files
with
203 additions
and
9 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
96 changes: 96 additions & 0 deletions
96
graphx/src/main/scala/org/apache/spark/graphx/lib/KCore.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
package org.apache.spark.graphx.lib | ||
|
||
import org.apache.spark.graphx._ | ||
import org.apache.spark._ | ||
import scala.math._ | ||
import org.apache.spark.SparkContext._ | ||
import scala.reflect.ClassTag | ||
|
||
object KCore extends Logging { | ||
/** | ||
* Compute the k-core decomposition of the graph for all k <= kmax. This | ||
* uses the iterative pruning algorithm discussed by Alvarez-Hamelin et al. | ||
* in K-Core Decomposition: a Tool For the Visualization of Large Scale Networks | ||
* (see <a href="http://arxiv.org/abs/cs/0504107">http://arxiv.org/abs/cs/0504107</a>). | ||
* | ||
* @tparam VD the vertex attribute type (discarded in the computation) | ||
* @tparam ED the edge attribute type (preserved in the computation) | ||
* | ||
* @param graph the graph for which to compute the connected components | ||
* @param kmax the maximum value of k to decompose the graph | ||
* | ||
* @return a graph where the vertex attribute is the minimum of | ||
* kmax or the highest value k for which that vertex was a member of | ||
* the k-core. | ||
* | ||
* @note This method has the advantage of returning not just a single kcore of the | ||
* graph but will yield all the cores for all k in [1, kmax]. | ||
*/ | ||
|
||
def run[VD: ClassTag, ED: ClassTag]( | ||
graph: Graph[VD, ED], | ||
kmax: Int, | ||
kmin: Int = 1) | ||
: Graph[Int, ED] = { | ||
|
||
// Graph[(Int, Boolean), ED] - boolean indicates whether it is active or not | ||
var g = graph.outerJoinVertices(graph.degrees)((vid, oldData, newData) => (newData.getOrElse(0), true)).cache | ||
val degrees = graph.degrees | ||
val numVertices = degrees.count | ||
// logWarning(s"Numvertices: $numVertices") | ||
// logWarning(s"degree sample: ${degrees.take(10).mkString(", ")}") | ||
// logWarning("degree distribution: " + degrees.map{ case (vid,data) => (data, 1)}.reduceByKey((_+_)).collect().mkString(", ")) | ||
// logWarning("degree distribution: " + degrees.map{ case (vid,data) => (data, 1)}.reduceByKey((_+_)).take(10).mkString(", ")) | ||
var curK = kmin | ||
while (curK <= kmax) { | ||
g = computeCurrentKCore(g, curK).cache | ||
val testK = curK | ||
val vCount = g.vertices.filter{ case (vid, (vd, _)) => vd >= testK}.count() | ||
val eCount = g.triplets.map{t => t.srcAttr._1 >= testK && t.dstAttr._1 >= testK }.count() | ||
logWarning(s"K=$curK, V=$vCount, E=$eCount") | ||
curK += 1 | ||
} | ||
g.mapVertices({ case (_, (k, _)) => k}) | ||
} | ||
|
||
def computeCurrentKCore[ED: ClassTag](graph: Graph[(Int, Boolean), ED], k: Int) = { | ||
logWarning(s"Computing kcore for k=$k") | ||
def sendMsg(et: EdgeTriplet[(Int, Boolean), ED]): Iterator[(VertexId, (Int, Boolean))] = { | ||
if (!et.srcAttr._2 || !et.dstAttr._2) { | ||
// if either vertex has already been turned off we do nothing | ||
Iterator.empty | ||
} else if (et.srcAttr._1 < k && et.dstAttr._1 < k) { | ||
// tell both vertices to turn off but don't need change count value | ||
Iterator((et.srcId, (0, false)), (et.dstId, (0, false))) | ||
} else if (et.srcAttr._1 < k) { | ||
// if src is being pruned, tell dst to subtract from vertex count but don't turn off | ||
Iterator((et.srcId, (0, false)), (et.dstId, (1, true))) | ||
} else if (et.dstAttr._1 < k) { | ||
// if dst is being pruned, tell src to subtract from vertex count but don't turn off | ||
Iterator((et.dstId, (0, false)), (et.srcId, (1, true))) | ||
} else { | ||
// no-op but keep these vertices active? | ||
// Iterator((et.srcId, (0, true)), (et.dstId, (0, true))) | ||
Iterator.empty | ||
} | ||
} | ||
|
||
// subtracts removed neighbors from neighbor count and tells vertex whether it was turned off or not | ||
def mergeMsg(m1: (Int, Boolean), m2: (Int, Boolean)): (Int, Boolean) = { | ||
(m1._1 + m2._1, m1._2 && m2._2) | ||
} | ||
|
||
def vProg(vid: VertexId, data: (Int, Boolean), update: (Int, Boolean)): (Int, Boolean) = { | ||
var newCount = data._1 | ||
var on = data._2 | ||
if (on) { | ||
newCount = max(k - 1, data._1 - update._1) | ||
on = update._2 | ||
} | ||
(newCount, on) | ||
} | ||
|
||
// Note that initial message should have no effect | ||
Pregel(graph, (0, true))(vProg, sendMsg, mergeMsg) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
44 changes: 44 additions & 0 deletions
44
graphx/src/test/scala/org/apache/spark/graphx/lib/KCoreSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
package org.apache.spark.graphx.lib | ||
|
||
import org.scalatest.FunSuite | ||
|
||
import org.apache.spark.SparkContext | ||
import org.apache.spark.SparkContext._ | ||
import org.apache.spark.graphx._ | ||
// import org.apache.spark.graphx.util.GraphGenerators | ||
import org.apache.spark.rdd._ | ||
|
||
|
||
class KCoreSuite extends FunSuite with LocalSparkContext { | ||
|
||
def createTriple(sid: VertexId, did: VertexId, sattr: Int, dattr: Int, eattr: Int): EdgeTriplet[Int,Int] = { | ||
val et = new EdgeTriplet[Int,Int] | ||
et.srcId = sid | ||
et.dstId = did | ||
et.srcAttr = sattr | ||
et.dstAttr = dattr | ||
et.attr = eattr | ||
et | ||
} | ||
|
||
def createKCoreEdges(): Seq[Edge[Int]] = { | ||
Seq(Edge(11,31), Edge(12,31), Edge(31,33), Edge(31,32), Edge(31,34), Edge(33,34), | ||
Edge(33,32), Edge(34,32), Edge(32,13), Edge(32,23), Edge(34,23), Edge(23,14), | ||
Edge(34,21), Edge(34,22), Edge(21,22)) | ||
} | ||
|
||
test("KCore") { | ||
withSpark { sc => | ||
val rawEdges = createKCoreEdges() | ||
val vertices = Set((11, 1), (12,1), (13,1), (14,1), (21,2), (22,2), (23,2), (31, 3), (32,3), (33,3), (34,3)) | ||
val graph = Graph.fromEdges(sc.parallelize(rawEdges), "a") | ||
val resultGraph = KCore.run(graph, 5) | ||
val resultVerts = resultGraph.vertices.collect.toSet | ||
assert(resultVerts === vertices) | ||
|
||
} | ||
} | ||
|
||
|
||
|
||
} |