You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gearpump.apache.org by ap...@apache.org on 2016/04/11 20:26:06 UTC

[04/50] incubator-gearpump git commit: fix #1935, add topological sort algorithm for graph with circles

fix #1935, add topological sort algorithm for graph with circles


Project: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/commit/55135fbb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/tree/55135fbb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/diff/55135fbb

Branch: refs/heads/master
Commit: 55135fbb459935064dd0f6fa0843601698dd71e3
Parents: 20c62e5
Author: pangolulu <gy...@gmail.com>
Authored: Thu Jan 28 23:32:06 2016 +0800
Committer: pangolulu <gy...@gmail.com>
Committed: Mon Feb 1 19:12:42 2016 +0800

----------------------------------------------------------------------
 .../src/main/scala/io/gearpump/util/Graph.scala | 88 ++++++++++++++++++++
 .../test/scala/io/gearpump/util/GraphSpec.scala | 17 ++++
 2 files changed, 105 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/55135fbb/core/src/main/scala/io/gearpump/util/Graph.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/io/gearpump/util/Graph.scala b/core/src/main/scala/io/gearpump/util/Graph.scala
index c3035d0..6bff9da 100644
--- a/core/src/main/scala/io/gearpump/util/Graph.scala
+++ b/core/src/main/scala/io/gearpump/util/Graph.scala
@@ -254,6 +254,94 @@ class Graph[N, E](vertexList: List[N], edgeList: List[(N, E, N)]) extends Serial
   }
 
   /**
+   * Return all circles in graph.
+   * The reference of this algorithm is:
+   * https://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
+   */
+  private def findCircles: mutable.MutableList[mutable.MutableList[N]] = {
+    val inStack = mutable.Map.empty[N, Boolean]
+    val stack = mutable.Stack[N]()
+    val indexMap = mutable.Map.empty[N, Int]
+    val lowLink = mutable.Map.empty[N, Int]
+    var index = 0
+
+    val circles = mutable.MutableList.empty[mutable.MutableList[N]]
+
+    def tarjan(node: N): Unit = {
+      indexMap(node) = index
+      lowLink(node) = index
+      index += 1
+      inStack(node) = true
+      stack.push(node)
+
+      outgoingEdgesOf(node).foreach {
+        edge => {
+          if (!indexMap.contains(edge._3)) {
+            tarjan(edge._3)
+            if (lowLink.get(edge._3).get < lowLink.get(node).get)
+              lowLink(node) = lowLink(edge._3)
+          } else {
+            if (inStack.get(edge._3).get && (indexMap.get(edge._3).get < lowLink.get(node).get))
+              lowLink(node) = indexMap(edge._3)
+          }
+        }
+      }
+
+      if (indexMap.get(node).get == lowLink.get(node).get) {
+        val circle = mutable.MutableList.empty[N]
+        var n = node
+        do {
+          n = stack.pop()
+          inStack(n) = false
+          circle += n
+        } while (n != node)
+        circles += circle
+      }
+    }
+
+    vertices.foreach {
+      node => {
+        if (!indexMap.contains(node)) tarjan(node)
+      }
+    }
+
+    circles
+  }
+
+  /**
+   * Return an iterator of vertex in topological order of graph with circles
+   * The node returned by Iterator is stable sorted.
+   * The reference of this algorithm is:
+   * http://www.drdobbs.com/database/topological-sorting/184410262
+   */
+  def topologicalOrderWithCirclesIterator: Iterator[N] = {
+    val circles = findCircles
+    val newGraph = Graph.empty[mutable.MutableList[N], E]
+    circles.foreach {
+      circle => {
+        newGraph.addVertex(circle)
+      }
+    }
+
+    for (circle1 <- circles; circle2 <- circles; if circle1 != circle2) yield {
+      for (node1 <- circle1; node2 <- circle2) yield {
+        var edges = outgoingEdgesOf(node1)
+        for (edge <- edges; if edge._3 == node2) yield {
+          newGraph.addEdge(circle1, edge._2, circle2)
+        }
+
+        edges = outgoingEdgesOf(node2)
+        for (edge <- edges; if edge._3 == node1) yield {
+          newGraph.addEdge(circle2, edge._2, circle1)
+        }
+      }
+    }
+
+    val topo = newGraph.topologicalOrderIterator
+    topo.flatMap(_.sortBy(_indexs(_)).iterator)
+  }
+
+  /**
    * check whether there is a loop
    */
   def hasCycle(): Boolean = {

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/55135fbb/core/src/test/scala/io/gearpump/util/GraphSpec.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/io/gearpump/util/GraphSpec.scala b/core/src/test/scala/io/gearpump/util/GraphSpec.scala
index 892350c..0028446 100644
--- a/core/src/test/scala/io/gearpump/util/GraphSpec.scala
+++ b/core/src/test/scala/io/gearpump/util/GraphSpec.scala
@@ -178,6 +178,23 @@ class GraphSpec extends PropSpec with PropertyChecks with Matchers {
     assert(graph.hasCycle())
   }
 
+  property("topologicalOrderIterator " +
+    "and topologicalOrderWithCirclesIterator method should return equal order of graph with no circle") {
+    val graph = Graph(1 ~> 2 ~> 3, 4 ~> 2, 2 ~> 5)
+    val topoNoCircles = graph.topologicalOrderIterator
+    val topoWithCircles = graph.topologicalOrderWithCirclesIterator
+
+    assert(topoNoCircles.zip(topoWithCircles).forall(x => x._1 == x._2))
+  }
+
+  property("Topological sort of graph with circles should work properly") {
+    val graph = Graph(0 ~> 1 ~> 3 ~> 4 ~> 6 ~> 5 ~> 7, 4 ~> 1, 1 ~> 2 ~> 4, 7 ~> 6, 8 ~> 2, 6 ~> 9, 4 ~> 10)
+    val topoWithCircles = graph.topologicalOrderWithCirclesIterator
+    val trueTopoWithCircles = Iterator[Int](0, 8, 1, 3, 4, 2, 6 ,5, 7, 10, 9)
+
+    assert(trueTopoWithCircles.zip(topoWithCircles).forall(x => x._1 == x._2))
+  }
+
   property("Duplicated edges detecting should work properly") {
     val graph = Graph.empty[String, String]
     val defaultEdge = "edge"