You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hugegraph.apache.org by ji...@apache.org on 2022/11/23 03:07:09 UTC

[incubator-hugegraph-computer] branch master updated: feat: support cluster coefficient algorithm (#108)

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

jin 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 8b7dbd66 feat: support cluster coefficient algorithm (#108)
8b7dbd66 is described below

commit 8b7dbd667a81672e4eaa0f763b865601cdf413b1
Author: imbajin <ji...@apache.org>
AuthorDate: Wed Nov 23 11:07:03 2022 +0800

    feat: support cluster coefficient algorithm (#108)
    
    better add more test case (multi increase graph)
    
    * adapt apache version v1.0.0
    
    Co-authored-by: imbajin <im...@users.noreply.github.com>
---
 checkstyle.xml                                     |   2 +-
 .../centrality/degree/DegreeCentrality.java        |  12 +-
 .../community/cc/ClusteringCoefficient.java        | 142 +++++++++++++++++++++
 .../community/cc/ClusteringCoefficientOutput.java  |  64 ++++++++++
 .../community/cc/ClusteringCoefficientParams.java  |  44 +++++++
 .../ClusteringCoefficientValue.java}               |  52 ++++----
 .../trianglecount/TriangleCountValue.java          |   6 +-
 .../hugegraph/computer/core/graph/value/Value.java |   2 +-
 .../computer/core/config/ComputerOptions.java      |   2 +-
 .../computer/algorithm/AlgorithmTestSuite.java     |   2 +
 .../community/cc/ClusteringCoefficientTest.java    | 134 +++++++++++++++++++
 11 files changed, 425 insertions(+), 37 deletions(-)

diff --git a/checkstyle.xml b/checkstyle.xml
index ae62eec2..96156a36 100644
--- a/checkstyle.xml
+++ b/checkstyle.xml
@@ -80,7 +80,7 @@
         <module name="OperatorWrap">
             <property name="option" value="eol"/>
         </module>
-        <!--检查指定标记的周围是否有空格,比如:if、for、while、synchoronized 等-->
+        <!--检查指定标记的周围是否有空格,比如:if、for、while、synchronized 等-->
         <module name="WhitespaceAround"/>
         <!--左圆括号之后和右圆括号之前是否需要有一个空格,不需要-->
         <module name="ParenPad"/>
diff --git a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/centrality/degree/DegreeCentrality.java b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/centrality/degree/DegreeCentrality.java
index 6b34884f..d5289a9d 100644
--- a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/centrality/degree/DegreeCentrality.java
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/centrality/degree/DegreeCentrality.java
@@ -37,8 +37,7 @@ import org.apache.hugegraph.util.NumericUtil;
 
 public class DegreeCentrality implements Computation<NullValue> {
 
-    public static final String OPTION_WEIGHT_PROPERTY =
-                               "degree_centrality.weight_property";
+    public static final String OPTION_WEIGHT_PROPERTY = "degree_centrality.weight_property";
 
     private boolean calculateByWeightProperty;
     private String weightProperty;
@@ -64,16 +63,13 @@ public class DegreeCentrality implements Computation<NullValue> {
              *  int the future;
              */
             double totalWeight = 0.0;
-            Iterator<Edge> edges = vertex.edges().iterator();
-            while (edges.hasNext()) {
-                Edge edge = edges.next();
+            for (Edge edge : vertex.edges()) {
                 double weight = weightValue(edge.property(this.weightProperty));
                 totalWeight += weight;
                 if (Double.isInfinite(totalWeight)) {
-                    throw new ComputerException("Calculate weight overflow," +
+                    throw new ComputerException("Calculate weight overflow, " +
                                                 "current is %s, edge '%s' " +
-                                                "is %s",
-                                                totalWeight, edge, weight);
+                                                "is %s", totalWeight, edge, weight);
                 }
             }
             vertex.value(new DoubleValue(totalWeight));
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
new file mode 100644
index 00000000..6269349f
--- /dev/null
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficient.java
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+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.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.vertex.Vertex;
+import com.baidu.hugegraph.computer.core.worker.Computation;
+import com.baidu.hugegraph.computer.core.worker.ComputationContext;
+
+/**
+ * ClusteringCoefficient(CC) algorithm could calculate local & the whole graph:
+ * 1. local cc: get triangles & degree for current vertex, calculate them
+ * 2. whole cc have 2 ways to get the result: (NOT SUPPORTED NOW)
+ * - sum all open & closed triangles in graph, and calculate the result
+ * - sum all local cc for each vertex, and use avg as the whole graph result
+ * <p>
+ * 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> {
+
+    @Override
+    public String name() {
+        return "clustering_coefficient";
+    }
+
+    @Override
+    public String category() {
+        return "community";
+    }
+
+    @Override
+    public void init(Config config) {
+        // Reuse triangle count later
+    }
+
+    @Override
+    public void compute0(ComputationContext context, Vertex vertex) {
+        IdList selfId = new IdList();
+        selfId.add(vertex.id());
+
+        context.sendMessageToAllEdgesIf(vertex, selfId, (ids, targetId) -> {
+            return !ids.get(0).equals(targetId);
+        });
+        vertex.value(new ClusteringCoefficientValue());
+    }
+
+    @Override
+    public void compute(ComputationContext context, Vertex vertex, Iterator<IdList> messages) {
+        Integer count = this.triangleCount(context, vertex, messages);
+        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
new file mode 100644
index 00000000..3f1c68b2
--- /dev/null
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientOutput.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.algorithm.community.cc;
+
+import org.apache.hugegraph.structure.constant.WriteType;
+
+import com.baidu.hugegraph.computer.core.graph.vertex.Vertex;
+import com.baidu.hugegraph.computer.core.output.hg.HugeGraphOutput;
+
+/**
+ * Offer 2 ways to output: write-back + hdfs-file(TODO)
+ */
+public class ClusteringCoefficientOutput extends HugeGraphOutput<Float> {
+
+    @Override
+    public String name() {
+        return "clustering_coefficient";
+    }
+
+    @Override
+    public void prepareSchema() {
+        this.client().schema().propertyKey(this.name())
+            .asFloat()
+            .writeType(WriteType.OLAP_RANGE)
+            .ifNotExist()
+            .create();
+    }
+
+    @Override
+    protected org.apache.hugegraph.structure.graph.Vertex constructHugeVertex(Vertex vertex) {
+        org.apache.hugegraph.structure.graph.Vertex hugeVertex =
+                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();
+        hugeVertex.property(this.name(), 2 * triangle / degree / (degree - 1));
+        return hugeVertex;
+    }
+
+    /* TODO: enhance it
+    @Override
+    protected Float value(Vertex vertex) {
+        float triangle = ((ClusteringCoefficientValue) vertex.value()).count();
+        int degree = ((ClusteringCoefficientValue) vertex.value()).idList().size();
+        return 2 * triangle / degree / (degree - 1);
+    }*/
+}
diff --git a/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientParams.java b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientParams.java
new file mode 100644
index 00000000..8a310469
--- /dev/null
+++ b/computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientParams.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.algorithm.community.cc;
+
+import java.util.Map;
+
+import com.baidu.hugegraph.computer.algorithm.AlgorithmParams;
+import com.baidu.hugegraph.computer.core.config.ComputerOptions;
+import com.baidu.hugegraph.computer.core.config.EdgeFrequency;
+import com.baidu.hugegraph.computer.core.graph.value.IdList;
+
+public class ClusteringCoefficientParams implements AlgorithmParams {
+
+    @Override
+    public void setAlgorithmParameters(Map<String, String> params) {
+        this.setIfAbsent(params, ComputerOptions.WORKER_COMPUTATION_CLASS,
+                         ClusteringCoefficient.class.getName());
+        this.setIfAbsent(params, ComputerOptions.ALGORITHM_MESSAGE_CLASS,
+                         IdList.class.getName());
+        this.setIfAbsent(params, ComputerOptions.ALGORITHM_RESULT_CLASS,
+                         ClusteringCoefficientValue.class.getName());
+        this.setIfAbsent(params, ComputerOptions.OUTPUT_CLASS,
+                         ClusteringCoefficientOutput.class.getName());
+        this.setIfAbsent(params, ComputerOptions.INPUT_EDGE_FREQ.name(),
+                         EdgeFrequency.SINGLE.name());
+    }
+}
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/cc/ClusteringCoefficientValue.java
similarity index 66%
copy from computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/trianglecount/TriangleCountValue.java
copy to computer-algorithm/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientValue.java
index b687b276..6892afd1 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/cc/ClusteringCoefficientValue.java
@@ -17,26 +17,29 @@
  * under the License.
  */
 
-package com.baidu.hugegraph.computer.algorithm.community.trianglecount;
+package com.baidu.hugegraph.computer.algorithm.community.cc;
 
 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.IntValue;
-import com.baidu.hugegraph.computer.core.graph.value.Value.CustomizeValue;
+import com.baidu.hugegraph.computer.core.graph.value.Value;
 import com.baidu.hugegraph.computer.core.io.RandomAccessInput;
 import com.baidu.hugegraph.computer.core.io.RandomAccessOutput;
 
-public class TriangleCountValue implements CustomizeValue<Integer> {
+/**
+ * TODO: We could reuse triangle's result to simplify it (and avoid logical differences)
+ */
+public class ClusteringCoefficientValue implements Value.CustomizeValue<Integer> {
 
     private IdList idList;
     private IntValue count;
+    private final IntValue degree;
 
-    public TriangleCountValue() {
+    public ClusteringCoefficientValue() {
         this.idList = new IdList();
         this.count = new IntValue();
+        this.degree = new IntValue();
     }
 
     public IdList idList() {
@@ -44,19 +47,32 @@ public class TriangleCountValue implements CustomizeValue<Integer> {
     }
 
     public long count() {
-        return this.count.longValue();
+        return this.count.value();
     }
 
-    public void count(int count) {
+    public void count(Integer count) {
         this.count.value(count);
     }
 
+    public int degree() {
+        return this.degree.value();
+    }
+
+    public void degree(Integer degree) {
+        this.degree.value(degree);
+    }
+
     @Override
-    public TriangleCountValue copy() {
-        TriangleCountValue triangleCountValue = new TriangleCountValue();
-        triangleCountValue.idList = this.idList.copy();
-        triangleCountValue.count = this.count.copy();
-        return triangleCountValue;
+    public ClusteringCoefficientValue copy() {
+        ClusteringCoefficientValue ccValue = new ClusteringCoefficientValue();
+        ccValue.idList = this.idList.copy();
+        ccValue.count = this.count.copy();
+        return ccValue;
+    }
+
+    @Override
+    public Integer value() {
+        return this.count.value();
     }
 
     @Override
@@ -73,14 +89,6 @@ public class TriangleCountValue implements CustomizeValue<Integer> {
 
     @Override
     public String toString() {
-        return new ToStringBuilder(this)
-                   .append("idList", this.idList)
-                   .append("count", this.count)
-                   .toString();
-    }
-
-    @Override
-    public Integer value() {
-        return this.count.value();
+        return String.valueOf(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 b687b276..91f1d0e4 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
@@ -73,10 +73,8 @@ public class TriangleCountValue implements CustomizeValue<Integer> {
 
     @Override
     public String toString() {
-        return new ToStringBuilder(this)
-                   .append("idList", this.idList)
-                   .append("count", this.count)
-                   .toString();
+        return new ToStringBuilder(this).append("idList", this.idList)
+                                        .append("count", this.count).toString();
     }
 
     @Override
diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/Value.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/Value.java
index 16f2c980..4f03a962 100644
--- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/Value.java
+++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/Value.java
@@ -81,7 +81,7 @@ public interface Value extends Writable, Readable, Comparable<Value> {
 
     /**
      * Value class with template parameter for composite subclass extension,
-     * A customize Value class of the algorithm may need to extend this class.
+     * A customized Value class of the algorithm may need to extend this class.
      */
     interface CustomizeValue<T> extends Tvalue<T> {
 
diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java
index f3594b4a..97d5e21c 100644
--- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java
+++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/config/ComputerOptions.java
@@ -61,7 +61,7 @@ public class ComputerOptions extends OptionHolder {
     public static final ConfigOption<Class<?>> ALGORITHM_PARAMS_CLASS =
             new ConfigOption<>(
                     "algorithm.params_class",
-                    "The class used to transfer algorithms's parameters " +
+                    "The class used to transfer algorithms' parameters " +
                     "before algorithm been run.",
                     disallowEmpty(),
                     Null.class
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 e6fb5d18..ddb170a2 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
@@ -27,6 +27,7 @@ import com.baidu.hugegraph.computer.algorithm.centrality.betweenness.Betweenness
 import com.baidu.hugegraph.computer.algorithm.centrality.closeness.ClosenessCentralityTest;
 import com.baidu.hugegraph.computer.algorithm.centrality.degree.DegreeCentralityTest;
 import com.baidu.hugegraph.computer.algorithm.centrality.pagerank.PageRankTest;
+import com.baidu.hugegraph.computer.algorithm.community.cc.ClusteringCoefficientTest;
 import com.baidu.hugegraph.computer.algorithm.community.kcore.KcoreTest;
 import com.baidu.hugegraph.computer.algorithm.community.lpa.LpaTest;
 import com.baidu.hugegraph.computer.algorithm.community.trianglecount.TriangleCountTest;
@@ -45,6 +46,7 @@ import org.apache.hugegraph.config.OptionSpace;
     TriangleCountTest.class,
     RingsDetectionTest.class,
     RingsDetectionWithFilterTest.class,
+    ClusteringCoefficientTest.class,
     ClosenessCentralityTest.class,
     BetweennessCentralityTest.class
 })
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
new file mode 100644
index 00000000..7fce5b5d
--- /dev/null
+++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/algorithm/community/cc/ClusteringCoefficientTest.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2017 HugeGraph Authors
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.baidu.hugegraph.computer.algorithm.community.cc;
+
+import java.util.Map;
+
+import org.apache.hugegraph.driver.GraphManager;
+import org.apache.hugegraph.driver.SchemaManager;
+import org.apache.hugegraph.structure.constant.T;
+import org.apache.hugegraph.structure.graph.Vertex;
+import org.apache.hugegraph.testutil.Assert;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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";
+    private static final String EDGE_LABEL = "tc_know";
+    private static final String PROPERTY_KEY = "tc_weight";
+
+    protected static final Map<String, Object> EXPECTED_RESULTS =
+              ImmutableMap.of("tc_A", 0.6666667F, "tc_B", 1.0F,
+                              "tc_C", 0.5F, "tc_D", 0.6666667F,
+                              "tc_E", 1.0F);
+
+    @BeforeClass
+    public static void setup() {
+        clearAll();
+
+        SchemaManager schema = client().schema();
+        schema.propertyKey(PROPERTY_KEY)
+              .asInt()
+              .ifNotExist()
+              .create();
+        schema.vertexLabel(VERTX_LABEL)
+              .properties(PROPERTY_KEY)
+              .useCustomizeStringId()
+              .ifNotExist()
+              .create();
+        schema.edgeLabel(EDGE_LABEL)
+              .sourceLabel(VERTX_LABEL)
+              .targetLabel(VERTX_LABEL)
+              .properties(PROPERTY_KEY)
+              .ifNotExist()
+              .create();
+
+        GraphManager graph = client().graph();
+        Vertex vA = graph.addVertex(T.LABEL, VERTX_LABEL, T.ID, "tc_A",
+                                    PROPERTY_KEY, 1);
+        Vertex vB = graph.addVertex(T.LABEL, VERTX_LABEL, T.ID, "tc_B",
+                                    PROPERTY_KEY, 1);
+        Vertex vC = graph.addVertex(T.LABEL, VERTX_LABEL, T.ID, "tc_C",
+                                    PROPERTY_KEY, 1);
+        Vertex vD = graph.addVertex(T.LABEL, VERTX_LABEL, T.ID, "tc_D",
+                                    PROPERTY_KEY, 1);
+        Vertex vE = graph.addVertex(T.LABEL, VERTX_LABEL, T.ID, "tc_E",
+                                    PROPERTY_KEY, 1);
+
+        vA.addEdge(EDGE_LABEL, vB, PROPERTY_KEY, 1);
+        vA.addEdge(EDGE_LABEL, vC, PROPERTY_KEY, 1);
+        vB.addEdge(EDGE_LABEL, vC, PROPERTY_KEY, 1);
+        vC.addEdge(EDGE_LABEL, vD, PROPERTY_KEY, 1);
+        vD.addEdge(EDGE_LABEL, vA, PROPERTY_KEY, 1);
+        vD.addEdge(EDGE_LABEL, vE, PROPERTY_KEY, 1);
+        vE.addEdge(EDGE_LABEL, vD, PROPERTY_KEY, 1);
+        vE.addEdge(EDGE_LABEL, vC, PROPERTY_KEY, 1);
+    }
+
+    @AfterClass
+    public static void teardown() {
+        clearAll();
+    }
+
+    @Test
+    public void testClusteringCoefficientValue() {
+        ClusteringCoefficientValue value = new ClusteringCoefficientValue();
+        value.count(10);
+        Assert.assertThrows(UnsupportedOperationException.class,
+                            () -> value.assign(null));
+        Assert.assertThrows(UnsupportedOperationException.class,
+                            () -> value.compareTo(new ClusteringCoefficientValue()));
+
+        ClusteringCoefficientValue copy = value.copy();
+        Assert.assertEquals(10, copy.count());
+        Assert.assertNotSame(value.idList(), copy.idList());
+
+        Assert.assertContains("10", value.toString());
+    }
+
+    @Test
+    public void testClusteringCoefficient() throws InterruptedException {
+        runAlgorithm(ClusteringCoefficientParams.class.getName(),
+                     ComputerOptions.OUTPUT_CLASS.name(),
+                     ClusteringCoefficientOutputTest.class.getName());
+    }
+
+    public static class ClusteringCoefficientOutputTest extends ClusteringCoefficientOutput {
+
+        @Override
+        public Vertex constructHugeVertex(
+               com.baidu.hugegraph.computer.core.graph.vertex.Vertex vertex) {
+            Vertex result = super.constructHugeVertex(vertex);
+            Float expected = (Float) EXPECTED_RESULTS.get(result.id());
+
+            if (expected != null) {
+                Assert.assertEquals(expected, result.property(super.name()));
+            }
+            return result;
+        }
+    }
+}