You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hugegraph.apache.org by gu...@apache.org on 2022/11/26 11:55:46 UTC

[incubator-hugegraph-computer] branch master updated: improve TriangleCount and ClusteringCoefficient (#201)

This is an automated email from the ASF dual-hosted git repository.

guoshoujing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hugegraph-computer.git


The following commit(s) were added to refs/heads/master by this push:
     new 474f6ec7 improve TriangleCount and ClusteringCoefficient (#201)
474f6ec7 is described below

commit 474f6ec704c926e3a3147867247c5764010c0708
Author: Cong Zhao <zh...@apache.org>
AuthorDate: Sat Nov 26 19:55:42 2022 +0800

    improve TriangleCount and ClusteringCoefficient (#201)
---
 .../community/cc/ClusteringCoefficient.java        | 69 ++--------------------
 .../community/cc/ClusteringCoefficientOutput.java  |  2 +-
 .../community/cc/ClusteringCoefficientValue.java   | 20 +++----
 .../community/trianglecount/TriangleCount.java     | 20 +++----
 .../trianglecount/TriangleCountValue.java          | 22 +++----
 .../hugegraph/computer/core/graph/value/IdSet.java |  5 ++
 .../computer/core/graph/value/ListValue.java       |  2 +-
 .../computer/algorithm/AlgorithmTestSuite.java     | 12 ----
 .../community/cc/ClusteringCoefficientTest.java    |  3 +-
 .../community/trianglecount/TriangleCountTest.java |  4 +-
 10 files changed, 45 insertions(+), 114 deletions(-)

diff --git a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficient.java b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficient.java
index 6269349f..9cec92ef 100644
--- a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficient.java
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficient.java
@@ -19,17 +19,13 @@
 
 package com.baidu.hugegraph.computer.algorithm.community.cc;
 
-import java.util.HashSet;
 import java.util.Iterator;
-import java.util.Set;
 
+import com.baidu.hugegraph.computer.algorithm.community.trianglecount.TriangleCount;
 import com.baidu.hugegraph.computer.core.config.Config;
-import com.baidu.hugegraph.computer.core.graph.edge.Edge;
-import com.baidu.hugegraph.computer.core.graph.edge.Edges;
-import com.baidu.hugegraph.computer.core.graph.id.Id;
 import com.baidu.hugegraph.computer.core.graph.value.IdList;
+import com.baidu.hugegraph.computer.core.graph.value.IdSet;
 import com.baidu.hugegraph.computer.core.graph.vertex.Vertex;
-import com.baidu.hugegraph.computer.core.worker.Computation;
 import com.baidu.hugegraph.computer.core.worker.ComputationContext;
 
 /**
@@ -42,15 +38,12 @@ import com.baidu.hugegraph.computer.core.worker.ComputationContext;
  * And we have 2 ways to count local cc:
  * 1. if we already saved the triangles in each vertex, we can calculate only
  * in superstep0/compute0 to get the result
- * 2. if we want recount the triangles result, we can choose:
- * - copy code from TriangleCount, then add extra logic
- * - reuse code in TriangleCount (need solve compatible problem - TODO)
  * <p>
  * The formula of local CC is: C(v) = 2T / Dv(Dv - 1)
  * v represents one vertex, T represents the triangles of current vertex,
  * D represents the degree of current vertex
  */
-public class ClusteringCoefficient implements Computation<IdList> {
+public class ClusteringCoefficient extends TriangleCount {
 
     @Override
     public String name() {
@@ -80,63 +73,11 @@ public class ClusteringCoefficient implements Computation<IdList> {
 
     @Override
     public void compute(ComputationContext context, Vertex vertex, Iterator<IdList> messages) {
-        Integer count = this.triangleCount(context, vertex, messages);
+        IdSet neighbors = ((ClusteringCoefficientValue) vertex.value()).idSet();
+        Integer count = super.triangleCount(context, vertex, messages, neighbors);
         if (count != null) {
             ((ClusteringCoefficientValue) vertex.value()).count(count);
             vertex.inactivate();
         }
     }
-
-    private Integer triangleCount(ComputationContext context, Vertex vertex,
-                                  Iterator<IdList> messages) {
-        IdList neighbors = ((ClusteringCoefficientValue) vertex.value()).idList();
-
-        if (context.superstep() == 1) {
-            // Collect outgoing neighbors
-            Set<Id> outNeighbors = getOutNeighbors(vertex);
-            neighbors.addAll(outNeighbors);
-
-            // Collect incoming neighbors
-            while (messages.hasNext()) {
-                IdList idList = messages.next();
-                assert idList.size() == 1;
-                Id inId = idList.get(0);
-                if (!outNeighbors.contains(inId)) {
-                    neighbors.add(inId);
-                }
-            }
-            // TODO: Save degree to vertex value here (optional)
-            // Send all neighbors to neighbors
-            for (Id targetId : neighbors.values()) {
-                context.sendMessage(targetId, neighbors);
-            }
-        } else if (context.superstep() == 2) {
-            int count = 0;
-
-            Set<Id> allNeighbors = new HashSet<>(neighbors.values());
-            while (messages.hasNext()) {
-                IdList twoDegreeNeighbors = messages.next();
-                for (Id twoDegreeNeighbor : twoDegreeNeighbors.values()) {
-                    if (allNeighbors.contains(twoDegreeNeighbor)) {
-                        count++;
-                    }
-                }
-            }
-
-            return count >> 1;
-        }
-        return null;
-    }
-
-    private static Set<Id> getOutNeighbors(Vertex vertex) {
-        Set<Id> outNeighbors = new HashSet<>();
-        Edges edges = vertex.edges();
-        for (Edge edge : edges) {
-            Id targetId = edge.targetId();
-            if (!vertex.id().equals(targetId)) {
-                outNeighbors.add(targetId);
-            }
-        }
-        return outNeighbors;
-    }
 }
diff --git a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientOutput.java b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientOutput.java
index 3f1c68b2..0d7ed78f 100644
--- a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientOutput.java
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientOutput.java
@@ -49,7 +49,7 @@ public class ClusteringCoefficientOutput extends HugeGraphOutput<Float> {
                 new org.apache.hugegraph.structure.graph.Vertex(null);
         hugeVertex.id(vertex.id().asObject());
         float triangle = ((ClusteringCoefficientValue) vertex.value()).count();
-        int degree = ((ClusteringCoefficientValue) vertex.value()).idList().size();
+        int degree = ((ClusteringCoefficientValue) vertex.value()).idSet().value().size();
         hugeVertex.property(this.name(), 2 * triangle / degree / (degree - 1));
         return hugeVertex;
     }
diff --git a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientValue.java b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientValue.java
index 6892afd1..16c9f776 100644
--- a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientValue.java
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientValue.java
@@ -21,7 +21,7 @@ package com.baidu.hugegraph.computer.algorithm.community.cc;
 
 import java.io.IOException;
 
-import com.baidu.hugegraph.computer.core.graph.value.IdList;
+import com.baidu.hugegraph.computer.core.graph.value.IdSet;
 import com.baidu.hugegraph.computer.core.graph.value.IntValue;
 import com.baidu.hugegraph.computer.core.graph.value.Value;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
@@ -32,22 +32,22 @@ import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
  */
 public class ClusteringCoefficientValue implements Value.CustomizeValue<Integer> {
 
-    private IdList idList;
+    private IdSet idSet;
     private IntValue count;
     private final IntValue degree;
 
     public ClusteringCoefficientValue() {
-        this.idList = new IdList();
+        this.idSet = new IdSet();
         this.count = new IntValue();
         this.degree = new IntValue();
     }
 
-    public IdList idList() {
-        return this.idList;
+    public IdSet idSet() {
+        return this.idSet;
     }
 
-    public long count() {
-        return this.count.value();
+    public int count() {
+        return this.count.intValue();
     }
 
     public void count(Integer count) {
@@ -65,7 +65,7 @@ public class ClusteringCoefficientValue implements Value.CustomizeValue<Integer>
     @Override
     public ClusteringCoefficientValue copy() {
         ClusteringCoefficientValue ccValue = new ClusteringCoefficientValue();
-        ccValue.idList = this.idList.copy();
+        ccValue.idSet = this.idSet.copy();
         ccValue.count = this.count.copy();
         return ccValue;
     }
@@ -77,13 +77,13 @@ public class ClusteringCoefficientValue implements Value.CustomizeValue<Integer>
 
     @Override
     public void read(RandomAccessInput in) throws IOException {
-        this.idList.read(in);
+        this.idSet.read(in);
         this.count.read(in);
     }
 
     @Override
     public void write(RandomAccessOutput out) throws IOException {
-        this.idList.write(out);
+        this.idSet.write(out);
         this.count.write(out);
     }
 
diff --git a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCount.java b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCount.java
index 7d3d0222..7fa082d0 100644
--- a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCount.java
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCount.java
@@ -19,7 +19,6 @@
 
 package com.baidu.hugegraph.computer.algorithm.community.trianglecount;
 
-
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
@@ -28,6 +27,7 @@ import com.baidu.hugegraph.computer.core.graph.edge.Edge;
 import com.baidu.hugegraph.computer.core.graph.edge.Edges;
 import com.baidu.hugegraph.computer.core.graph.id.Id;
 import com.baidu.hugegraph.computer.core.graph.value.IdList;
+import com.baidu.hugegraph.computer.core.graph.value.IdSet;
 import com.baidu.hugegraph.computer.core.graph.vertex.Vertex;
 import com.baidu.hugegraph.computer.core.worker.Computation;
 import com.baidu.hugegraph.computer.core.worker.ComputationContext;
@@ -46,11 +46,11 @@ public class TriangleCount implements Computation<IdList> {
 
     @Override
     public void compute0(ComputationContext context, Vertex vertex) {
-        IdList selfId = new IdList();
+        IdSet selfId = new IdSet();
         selfId.add(vertex.id());
 
         context.sendMessageToAllEdgesIf(vertex, selfId, (ids, targetId) -> {
-            return !ids.get(0).equals(targetId);
+            return !vertex.id().equals(targetId);
         });
         vertex.value(new TriangleCountValue());
     }
@@ -58,17 +58,16 @@ public class TriangleCount implements Computation<IdList> {
     @Override
     public void compute(ComputationContext context, Vertex vertex,
                         Iterator<IdList> messages) {
-        Integer count = this.triangleCount(context, vertex, messages);
+        IdSet neighbors = ((TriangleCountValue) vertex.value()).idSet();
+        Integer count = this.triangleCount(context, vertex, messages, neighbors);
         if (count != null) {
             ((TriangleCountValue) vertex.value()).count(count);
             vertex.inactivate();
         }
     }
 
-    private Integer triangleCount(ComputationContext context, Vertex vertex,
-                                  Iterator<IdList> messages) {
-        IdList neighbors = ((TriangleCountValue) vertex.value()).idList();
-
+    protected Integer triangleCount(ComputationContext context, Vertex vertex,
+                                    Iterator<IdList> messages, IdSet neighbors) {
         if (context.superstep() == 1) {
             // Collect outgoing neighbors
             Set<Id> outNeighbors = getOutNeighbors(vertex);
@@ -85,17 +84,16 @@ public class TriangleCount implements Computation<IdList> {
             }
 
             // Send all neighbors to neighbors
-            for (Id targetId : neighbors.values()) {
+            for (Id targetId : neighbors.value()) {
                 context.sendMessage(targetId, neighbors);
             }
         } else if (context.superstep() == 2) {
             int count = 0;
 
-            Set<Id> allNeighbors = new HashSet<>(neighbors.values());
             while (messages.hasNext()) {
                 IdList twoDegreeNeighbors = messages.next();
                 for (Id twoDegreeNeighbor : twoDegreeNeighbors.values()) {
-                    if (allNeighbors.contains(twoDegreeNeighbor)) {
+                    if (neighbors.contains(twoDegreeNeighbor)) {
                         count++;
                     }
                 }
diff --git a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountValue.java b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountValue.java
index 91f1d0e4..b50007d0 100644
--- a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountValue.java
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountValue.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.commons.lang3.builder.ToStringBuilder;
 
-import com.baidu.hugegraph.computer.core.graph.value.IdList;
+import com.baidu.hugegraph.computer.core.graph.value.IdSet;
 import com.baidu.hugegraph.computer.core.graph.value.IntValue;
 import com.baidu.hugegraph.computer.core.graph.value.Value.CustomizeValue;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
@@ -31,20 +31,20 @@ import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
 
 public class TriangleCountValue implements CustomizeValue<Integer> {
 
-    private IdList idList;
+    private IdSet idSet;
     private IntValue count;
 
     public TriangleCountValue() {
-        this.idList = new IdList();
+        this.idSet = new IdSet();
         this.count = new IntValue();
     }
 
-    public IdList idList() {
-        return this.idList;
+    public IdSet idSet() {
+        return this.idSet;
     }
 
-    public long count() {
-        return this.count.longValue();
+    public int count() {
+        return this.count.intValue();
     }
 
     public void count(int count) {
@@ -54,26 +54,26 @@ public class TriangleCountValue implements CustomizeValue<Integer> {
     @Override
     public TriangleCountValue copy() {
         TriangleCountValue triangleCountValue = new TriangleCountValue();
-        triangleCountValue.idList = this.idList.copy();
+        triangleCountValue.idSet = this.idSet.copy();
         triangleCountValue.count = this.count.copy();
         return triangleCountValue;
     }
 
     @Override
     public void read(RandomAccessInput in) throws IOException {
-        this.idList.read(in);
+        this.idSet.read(in);
         this.count.read(in);
     }
 
     @Override
     public void write(RandomAccessOutput out) throws IOException {
-        this.idList.write(out);
+        this.idSet.write(out);
         this.count.write(out);
     }
 
     @Override
     public String toString() {
-        return new ToStringBuilder(this).append("idList", this.idList)
+        return new ToStringBuilder(this).append("idSet", this.idSet)
                                         .append("count", this.count).toString();
     }
 
diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdSet.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdSet.java
index 469979dc..acf704ad 100644
--- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdSet.java
+++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdSet.java
@@ -20,6 +20,7 @@
 package com.baidu.hugegraph.computer.core.graph.value;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
 
@@ -48,6 +49,10 @@ public class IdSet implements Tvalue<Set<Id>> {
         this.values.addAll(other.values);
     }
 
+    public void addAll(Collection<Id> other) {
+        this.values.addAll(other);
+    }
+
     public boolean contains(Id id) {
         return this.values.contains(id);
     }
diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java
index 1b78dc74..230fee92 100644
--- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java
+++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java
@@ -29,6 +29,7 @@ import java.util.NoSuchElementException;
 
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.ListUtils;
+import org.apache.hugegraph.util.E;
 
 import com.baidu.hugegraph.computer.core.common.ComputerContext;
 import com.baidu.hugegraph.computer.core.common.SerialEnum;
@@ -36,7 +37,6 @@ import com.baidu.hugegraph.computer.core.graph.GraphFactory;
 import com.baidu.hugegraph.computer.core.graph.value.Value.Tvalue;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
-import org.apache.hugegraph.util.E;
 
 public class ListValue<T extends Tvalue<?>> implements Tvalue<List<Object>> {
 
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/AlgorithmTestSuite.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/AlgorithmTestSuite.java
index ddb170a2..6ffa363c 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/AlgorithmTestSuite.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/AlgorithmTestSuite.java
@@ -19,7 +19,6 @@
 
 package com.baidu.hugegraph.computer.algorithm;
 
-import org.junit.BeforeClass;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
@@ -34,7 +33,6 @@ import com.baidu.hugegraph.computer.algorithm.community.trianglecount.TriangleCo
 import com.baidu.hugegraph.computer.algorithm.community.wcc.WccTest;
 import com.baidu.hugegraph.computer.algorithm.path.rings.RingsDetectionTest;
 import com.baidu.hugegraph.computer.algorithm.path.rings.RingsDetectionWithFilterTest;
-import org.apache.hugegraph.config.OptionSpace;
 
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
@@ -51,14 +49,4 @@ import org.apache.hugegraph.config.OptionSpace;
     BetweennessCentralityTest.class
 })
 public class AlgorithmTestSuite {
-
-    @BeforeClass
-    public static void setup() throws ClassNotFoundException {
-        // Don't forget to register options
-        OptionSpace.register("computer",
-                             "com.baidu.hugegraph.computer.core.config." +
-                             "ComputerOptions");
-        OptionSpace.register("computer-rpc",
-                             "org.apache.hugegraph.config.RpcOptions");
-    }
 }
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientTest.java
index 7fce5b5d..4866585c 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientTest.java
@@ -34,7 +34,6 @@ import com.baidu.hugegraph.computer.algorithm.AlgorithmTestBase;
 import com.baidu.hugegraph.computer.core.config.ComputerOptions;
 import com.google.common.collect.ImmutableMap;
 
-
 public class ClusteringCoefficientTest extends AlgorithmTestBase {
 
     private static final String VERTX_LABEL = "tc_user";
@@ -105,7 +104,7 @@ public class ClusteringCoefficientTest extends AlgorithmTestBase {
 
         ClusteringCoefficientValue copy = value.copy();
         Assert.assertEquals(10, copy.count());
-        Assert.assertNotSame(value.idList(), copy.idList());
+        Assert.assertNotSame(value.idSet(), copy.idSet());
 
         Assert.assertContains("10", value.toString());
     }
diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountTest.java
index 3c33ee1e..f6e103c0 100644
--- a/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountTest.java
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountTest.java
@@ -113,7 +113,7 @@ public class TriangleCountTest extends AlgorithmTestBase {
 
         TriangleCountValue copy = value.copy();
         Assert.assertEquals(10, copy.count());
-        Assert.assertNotSame(value.idList(), copy.idList());
+        Assert.assertNotSame(value.idSet(), copy.idSet());
 
         Assert.assertContains("10", value.toString());
     }
@@ -123,7 +123,7 @@ public class TriangleCountTest extends AlgorithmTestBase {
         @Override
         public Integer value(com.baidu.hugegraph.computer.core.graph.vertex.Vertex vertex) {
             Integer value = super.value(vertex);
-            Integer expected = EXPECTED_RESULTS.get(vertex.id());
+            Integer expected = EXPECTED_RESULTS.get(vertex.id().string());
             if (expected != null) {
                 Assert.assertEquals(expected, value);
             }