diff options
author | Takeshi Yamamuro <linguin.m.s@gmail.com> | 2015-01-08 09:55:12 -0800 |
---|---|---|
committer | Ankur Dave <ankurdave@gmail.com> | 2015-01-08 09:55:12 -0800 |
commit | f825e193f3357e60949bf4c0174675d0d1a40988 (patch) | |
tree | 83b65888da042642717aca572c584f70ba215bbc /graphx/src/test/scala/org/apache | |
parent | 8d45834debc6986e61831d0d6e982d5528dccc51 (diff) | |
download | spark-f825e193f3357e60949bf4c0174675d0d1a40988.tar.gz spark-f825e193f3357e60949bf4c0174675d0d1a40988.tar.bz2 spark-f825e193f3357e60949bf4c0174675d0d1a40988.zip |
[SPARK-4917] Add a function to convert into a graph with canonical edges in GraphOps
Convert bi-directional edges into uni-directional ones instead of 'canonicalOrientation' in GraphLoader.edgeListFile.
This function is useful when a graph is loaded as it is and then is transformed into one with canonical edges.
It rewrites the vertex ids of edges so that srcIds are bigger than dstIds, and merges the duplicated edges.
Author: Takeshi Yamamuro <linguin.m.s@gmail.com>
Closes #3760 from maropu/ConvertToCanonicalEdgesSpike and squashes the following commits:
7f8b580 [Takeshi Yamamuro] Add a function to convert into a graph with canonical edges in GraphOps
Diffstat (limited to 'graphx/src/test/scala/org/apache')
-rw-r--r-- | graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala | 15 |
1 files changed, 15 insertions, 0 deletions
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index ea94d4accb..9bc8007ce4 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -79,6 +79,21 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { } } + test ("convertToCanonicalEdges") { + withSpark { sc => + val vertices = + sc.parallelize(Seq[(VertexId, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val edges = + sc.parallelize(Seq(Edge(1, 2, 1), Edge(2, 1, 1), Edge(3, 2, 2))) + val g: Graph[String, Int] = Graph(vertices, edges) + + val g1 = g.convertToCanonicalEdges() + + val e = g1.edges.collect().toSet + assert(e === Set(Edge(1, 2, 1), Edge(2, 3, 2))) + } + } + test("collectEdgesCycleDirectionOut") { withSpark { sc => val graph = getCycleGraph(sc, 100) |