From ab1564a02ac53174e09ee92121ba12d84c188c2a Mon Sep 17 00:00:00 2001 From: zhoney Date: Wed, 2 Sep 2020 23:15:35 +0800 Subject: [PATCH 01/20] Add customized kout/kneighbor, template path, multi-node-shortest-path and jaccard similar * fix OLTP algorithm not check if source/target vertex exist (#1156) * support customized kout and kneighbor * support multi node shortest path API * support template paths api * support jaccard similars find oltp api * use multi-thread to accelerate customized kout/kneighbor, multi-node-shortest-path, jaccard-similar and template-paths * big depth and both direction use multi threads, otherwise single threead * fix shortest path api NLP Change-Id: I7eb4ceaccdc4a6b2a4a7b944edc83dff64a98f5d --- .../api/traversers/AllShortestPathsAPI.java | 8 +- .../traversers/CustomizedCrosspointsAPI.java | 4 +- .../traversers/CustomizedKneighborAPI.java | 145 ++++++ .../api/traversers/CustomizedKoutAPI.java | 153 +++++++ .../api/traversers/CustomizedPathsAPI.java | 4 +- .../api/traversers/FusiformSimilarityAPI.java | 4 +- .../api/traversers/JaccardSimilarAPI.java | 105 +++++ .../traversers/MultiNodeShortestPathAPI.java | 126 ++++++ .../api/traversers/ShortestPathAPI.java | 8 +- .../api/traversers/TemplatePathAPI.java | 139 ++++++ .../api/traversers/TraverserAPI.java | 61 +++ .../{SourceVertices.java => Vertices.java} | 4 +- .../hugegraph/serializer/JsonSerializer.java | 21 + .../hugegraph/serializer/Serializer.java | 7 + .../baidu/hugegraph/config/CoreOptions.java | 8 + .../CustomizedKneighborTraverser.java | 113 +++++ .../algorithm/CustomizedKoutTraverser.java | 135 ++++++ .../traversal/algorithm/EdgeStep.java | 6 +- .../traversal/algorithm/HugeTraverser.java | 68 ++- .../algorithm/JaccardSimilarTraverser.java | 182 ++++++++ .../MultiNodeShortestPathTraverser.java | 130 ++++++ .../algorithm/ShortestPathTraverser.java | 43 +- .../algorithm/TemplatePathsTraverser.java | 418 ++++++++++++++++++ .../traversal/algorithm/TpTraverser.java | 142 ++++++ 24 files changed, 2004 insertions(+), 30 deletions(-) create mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java create mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java create mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java create mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/MultiNodeShortestPathAPI.java create mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java create mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java rename hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/{SourceVertices.java => Vertices.java} (97%) create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java index 1f88ad4654..62859abdc4 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java @@ -19,6 +19,8 @@ package com.baidu.hugegraph.api.traversers; +import java.util.List; + import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; @@ -45,6 +47,7 @@ import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.Log; import com.codahale.metrics.annotation.Timed; +import com.google.common.collect.ImmutableList; @Path("graphs/{graph}/traversers/allshortestpaths") @Singleton @@ -81,9 +84,10 @@ public String get(@Context GraphManager manager, HugeGraph g = graph(manager, graph); ShortestPathTraverser traverser = new ShortestPathTraverser(g); + List edgeLabels = ImmutableList.of(edgeLabel); HugeTraverser.PathSet paths = traverser.allShortestPaths( - sourceId, targetId, dir, edgeLabel, depth, - degree, skipDegree, capacity); + sourceId, targetId, dir, edgeLabels, + depth, degree, skipDegree, capacity); return manager.serializer(g).writePaths("paths", paths, false); } } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedCrosspointsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedCrosspointsAPI.java index 26a4cf00de..293905a9fd 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedCrosspointsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedCrosspointsAPI.java @@ -84,7 +84,7 @@ public String post(@Context GraphManager manager, request.capacity, request.limit); HugeGraph g = graph(manager, graph); - Iterator sources = request.sources.sourcesVertices(g); + Iterator sources = request.sources.vertices(g); List patterns; patterns = pathPatterns(g, request); @@ -132,7 +132,7 @@ public String post(@Context GraphManager manager, private static class CrosspointsRequest { @JsonProperty("sources") - public SourceVertices sources; + public Vertices sources; @JsonProperty("path_patterns") public List pathPatterns; @JsonProperty("capacity") diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java new file mode 100644 index 0000000000..b51a5c79a6 --- /dev/null +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java @@ -0,0 +1,145 @@ +/* + * 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.api.traversers; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import javax.inject.Singleton; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; + +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.slf4j.Logger; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.query.QueryResults; +import com.baidu.hugegraph.core.GraphManager; +import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.CustomizedKneighborTraverser; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; +import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.Log; +import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.*; + +@Path("graphs/{graph}/traversers/customizedkneighbor") +@Singleton +public class CustomizedKneighborAPI extends TraverserAPI { + + private static final Logger LOG = Log.logger(RestServer.class); + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.source, + "The source of request can't be null"); + E.checkArgument(request.step != null, + "The steps of request can't be null"); + if (request.countOnly) { + E.checkArgument(!request.withVertex && !request.withPath, + "Can't return vertex or path when count only"); + } + + LOG.debug("Graph [{}] get customized kneighbor from source vertex " + + "'{}', with step '{}', limit '{}', count_only '{}', " + + "with_vertex '{}' and with_path '{}'", + graph, request.source, request.step, request.limit, + request.countOnly, request.withVertex, request.withPath); + + HugeGraph g = graph(manager, graph); + Id sourceId = HugeVertex.getIdValue(request.source); + + EdgeStep step = step(g, request.step); + + CustomizedKneighborTraverser traverser = + new CustomizedKneighborTraverser(g); + Set results = traverser.customizedKneighbor(sourceId, step, + request.maxDepth, + request.limit); + Set neighbors = new HashSet<>(); + for (Node node : results) { + neighbors.add(node.id()); + } + + List paths = new ArrayList<>(); + if (request.withPath) { + for (Node node : results) { + paths.add(new HugeTraverser.Path(node.path())); + } + } + Iterator iter = QueryResults.emptyIterator(); + if (request.withVertex) { + Set ids = new HashSet<>(); + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } + if (!ids.isEmpty()) { + iter = g.vertices(ids.toArray()); + } + } + return manager.serializer(g).writeNodesWithPath("kneighbor", neighbors, + paths, iter, request.countOnly); + } + + private static class Request { + + @JsonProperty("source") + public Object source; + @JsonProperty("step") + public TraverserAPI.Step step; + @JsonProperty("max_depth") + public int maxDepth; + @JsonProperty("limit") + public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); + @JsonProperty("count_only") + public boolean countOnly = false; + @JsonProperty("with_vertex") + public boolean withVertex = false; + @JsonProperty("with_path") + public boolean withPath = false; + + @Override + public String toString() { + return String.format("PathRequest{source=%s,step=%s,maxDepth=%s" + + "limit=%s,countOnly=%s,withVertex=%s," + + "withPath=%s}", this.source, this.step, + this.maxDepth, this.limit, this.countOnly, + this.withVertex, this.withPath); + } + } +} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java new file mode 100644 index 0000000000..bcf1e1519f --- /dev/null +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java @@ -0,0 +1,153 @@ +/* + * 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.api.traversers; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import javax.inject.Singleton; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; + +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.slf4j.Logger; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.query.QueryResults; +import com.baidu.hugegraph.core.GraphManager; +import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.CustomizedKoutTraverser; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; +import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.Log; +import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.*; + +@Path("graphs/{graph}/traversers/customizedkout") +@Singleton +public class CustomizedKoutAPI extends TraverserAPI { + + private static final Logger LOG = Log.logger(RestServer.class); + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.source, + "The source of request can't be null"); + E.checkArgument(request.step != null, + "The steps of request can't be null"); + if (request.countOnly) { + E.checkArgument(!request.withVertex && !request.withPath, + "Can't return vertex or path when count only"); + } + + LOG.debug("Graph [{}] get customized kout from source vertex '{}', " + + "with step '{}', max_depth '{}', nearest '{}', " + + "count_only '{}', capacity '{}', limit '{}', " + + "with_vertex '{}' and with_path '{}'", + graph, request.source, request.step, request.maxDepth, + request.nearest, request.countOnly, request.capacity, + request.limit, request.withVertex, request.withPath); + + HugeGraph g = graph(manager, graph); + Id sourceId = HugeVertex.getIdValue(request.source); + + EdgeStep step = step(g, request.step); + + CustomizedKoutTraverser traverser = new CustomizedKoutTraverser(g); + Set results = traverser.customizedKout(sourceId, step, + request.maxDepth, + request.nearest, + request.capacity, + request.limit); + Set neighbors = new HashSet<>(); + for (Node node : results) { + neighbors.add(node.id()); + } + + List paths = new ArrayList<>(); + if (request.withPath) { + for (Node node : results) { + paths.add(new HugeTraverser.Path(node.path())); + } + } + Iterator iter = QueryResults.emptyIterator(); + if (request.withVertex) { + Set ids = new HashSet<>(); + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } + if (!ids.isEmpty()) { + iter = g.vertices(ids.toArray()); + } + } + return manager.serializer(g).writeNodesWithPath("kout", neighbors, + paths, iter, request.countOnly); + } + + private static class Request { + + @JsonProperty("source") + public Object source; + @JsonProperty("step") + public TraverserAPI.Step step; + @JsonProperty("max_depth") + public int maxDepth; + @JsonProperty("nearest") + public boolean nearest = true; + @JsonProperty("count_only") + public boolean countOnly = false; + @JsonProperty("capacity") + public long capacity = Long.valueOf(DEFAULT_CAPACITY); + @JsonProperty("limit") + public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); + @JsonProperty("with_vertex") + public boolean withVertex = false; + @JsonProperty("with_path") + public boolean withPath = false; + + @Override + public String toString() { + return String.format("KoutRequest{source=%s,step=%s,maxDepth=%s" + + "nearest=%s,countOnly=%s,capacity=%s," + + "limit=%s,withVertex=%s,withPath=%s}", + this.source, this.step, this.maxDepth, + this.nearest, this.countOnly, this.capacity, + this.limit, this.withVertex, this.withPath); + } + } +} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedPathsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedPathsAPI.java index 8dfac69cd2..e49f808e0c 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedPathsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedPathsAPI.java @@ -86,7 +86,7 @@ public String post(@Context GraphManager manager, request.withVertex); HugeGraph g = graph(manager, graph); - Iterator sources = request.sources.sourcesVertices(g); + Iterator sources = request.sources.vertices(g); List steps = step(g, request); boolean sorted = request.sortBy != SortBy.NONE; @@ -129,7 +129,7 @@ private static List step(HugeGraph graph, private static class PathRequest { @JsonProperty("sources") - public SourceVertices sources; + public Vertices sources; @JsonProperty("steps") public List steps; @JsonProperty("sort_by") diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/FusiformSimilarityAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/FusiformSimilarityAPI.java index 0a56f56b82..e6ef2858b7 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/FusiformSimilarityAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/FusiformSimilarityAPI.java @@ -97,7 +97,7 @@ public String post(@Context GraphManager manager, request.groupProperty, request.minGroups); HugeGraph g = graph(manager, graph); - Iterator sources = request.sources.sourcesVertices(g); + Iterator sources = request.sources.vertices(g); E.checkArgument(sources != null && sources.hasNext(), "The source vertices can't be empty"); EdgeLabel edgeLabel = request.label == null ? @@ -125,7 +125,7 @@ public String post(@Context GraphManager manager, private static class FusiformSimilarityRequest { @JsonProperty("sources") - public SourceVertices sources; + public Vertices sources; @JsonProperty("label") public String label; @JsonProperty("direction") diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java new file mode 100644 index 0000000000..5fc54796b0 --- /dev/null +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java @@ -0,0 +1,105 @@ +/* + * 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.api.traversers; + +import java.util.Map; + +import javax.inject.Singleton; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; + +import org.slf4j.Logger; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.core.GraphManager; +import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.JaccardSimilarTraverser; +import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.Log; +import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser + .DEFAULT_CAPACITY; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_LIMIT; + +@Path("graphs/{graph}/traversers/jaccardsimilar") +@Singleton +public class JaccardSimilarAPI extends TraverserAPI { + + private static final Logger LOG = Log.logger(RestServer.class); + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.vertex, + "The source vertex of request can't be null"); + E.checkArgument(request.step != null, + "The steps of request can't be null"); + E.checkArgument(request.top >= 0, + "The top must be >= 0, but got: %s", request.top); + + LOG.debug("Graph [{}] get jaccard similars from source vertex '{}', " + + "with step '{}', top '{}' and capacity '{}'", + graph, request.vertex, request.step, request.top); + + HugeGraph g = graph(manager, graph); + Id sourceId = HugeVertex.getIdValue(request.vertex); + + EdgeStep step = step(g, request.step); + + JaccardSimilarTraverser traverser = new JaccardSimilarTraverser(g); + Map results = traverser.jaccardSimilars(sourceId, step, + request.top, + request.capacity); + return manager.serializer(g).writeMap(results); + } + + private static class Request { + + @JsonProperty("vertex") + public Object vertex; + @JsonProperty("step") + public TraverserAPI.Step step; + @JsonProperty("top") + public int top = Integer.valueOf(DEFAULT_LIMIT); + @JsonProperty("capacity") + public long capacity = Long.valueOf(DEFAULT_CAPACITY); + + @Override + public String toString() { + return String.format("Request{vertex=%s,step=%s,top=%s," + + "capacity=%s}", this.vertex, this.step, + this.top, this.capacity); + } + } +} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/MultiNodeShortestPathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/MultiNodeShortestPathAPI.java new file mode 100644 index 0000000000..c3cebec720 --- /dev/null +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/MultiNodeShortestPathAPI.java @@ -0,0 +1,126 @@ +/* + * 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.api.traversers; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import javax.inject.Singleton; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; + +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.slf4j.Logger; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.query.QueryResults; +import com.baidu.hugegraph.core.GraphManager; +import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; +import com.baidu.hugegraph.traversal.algorithm.MultiNodeShortestPathTraverser; +import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.Log; +import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; + +@Path("graphs/{graph}/traversers/multinodeshortestpath") +@Singleton +public class MultiNodeShortestPathAPI extends TraverserAPI { + + private static final Logger LOG = Log.logger(RestServer.class); + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.vertices, + "The vertices of request can't be null"); + E.checkArgument(request.step != null, + "The steps of request can't be null"); + + LOG.debug("Graph [{}] get multiple node shortest path from " + + "vertices '{}', with step '{}', max_depth '{}', capacity " + + "'{}' and with_vertex '{}'", + graph, request.vertices, request.step, request.maxDepth, + request.capacity, request.withVertex); + + HugeGraph g = graph(manager, graph); + Iterator vertices = request.vertices.vertices(g); + + EdgeStep step = step(g, request.step); + + MultiNodeShortestPathTraverser traverser = + new MultiNodeShortestPathTraverser(g); + List paths; + paths = traverser.multiNodeShortestPath(vertices, step, + request.maxDepth, + request.capacity); + + if (!request.withVertex) { + return manager.serializer(g).writePaths("paths", paths, false); + } + + Set ids = new HashSet<>(); + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } + Iterator iter = QueryResults.emptyIterator(); + if (!ids.isEmpty()) { + iter = g.vertices(ids.toArray()); + } + return manager.serializer(g).writePaths("paths", paths, false, iter); + } + + private static class Request { + + @JsonProperty("vertices") + public Vertices vertices; + @JsonProperty("step") + public Step step; + @JsonProperty("max_depth") + public int maxDepth; + @JsonProperty("capacity") + public long capacity = Long.valueOf(DEFAULT_CAPACITY); + @JsonProperty("with_vertex") + public boolean withVertex = false; + + @Override + public String toString() { + return String.format("Request{vertices=%s,step=%s,maxDepth=%s" + + "capacity=%s,withVertex=%s}", + this.vertices, this.step, this.maxDepth, + this.capacity, this.withVertex); + } + } +} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/ShortestPathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/ShortestPathAPI.java index ab10da79cd..fa97a93e23 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/ShortestPathAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/ShortestPathAPI.java @@ -19,6 +19,8 @@ package com.baidu.hugegraph.api.traversers; +import java.util.List; + import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; @@ -45,6 +47,7 @@ import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.Log; import com.codahale.metrics.annotation.Timed; +import com.google.common.collect.ImmutableList; @Path("graphs/{graph}/traversers/shortestpath") @Singleton @@ -81,8 +84,11 @@ public String get(@Context GraphManager manager, HugeGraph g = graph(manager, graph); ShortestPathTraverser traverser = new ShortestPathTraverser(g); + + List edgeLabels = edgeLabel == null ? ImmutableList.of() : + ImmutableList.of(edgeLabel); HugeTraverser.Path path = traverser.shortestPath(sourceId, targetId, - dir, edgeLabel, depth, + dir, edgeLabels, depth, degree, skipDegree, capacity); return manager.serializer(g).writeList("path", path.vertices()); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java new file mode 100644 index 0000000000..f53d34cfe5 --- /dev/null +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java @@ -0,0 +1,139 @@ +/* + * 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.api.traversers; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import javax.inject.Singleton; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; + +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.slf4j.Logger; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.query.QueryResults; +import com.baidu.hugegraph.core.GraphManager; +import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; +import com.baidu.hugegraph.traversal.algorithm.TemplatePathsTraverser; +import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.Log; +import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_PATHS_LIMIT; + +@Path("graphs/{graph}/traversers/templatepaths") +@Singleton +public class TemplatePathAPI extends TraverserAPI { + + private static final Logger LOG = Log.logger(RestServer.class); + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.sources, + "The sources of request can't be null"); + E.checkArgumentNotNull(request.targets, + "The targets of request can't be null"); + E.checkArgument(request.steps != null && !request.steps.isEmpty(), + "The steps of request can't be empty"); + + LOG.debug("Graph [{}] get template paths from source vertices '{}', " + + "target vertices '{}', with steps '{}', " + + "capacity '{}', limit '{}' and with_vertex '{}'", + graph, request.sources, request.targets, request.steps, + request.capacity, request.limit, request.withVertex); + + HugeGraph g = graph(manager, graph); + Iterator sources = request.sources.vertices(g); + Iterator targets = request.targets.vertices(g); + List steps = steps(g, request.steps); + + TemplatePathsTraverser traverser = new TemplatePathsTraverser(g); + HugeTraverser.PathSet pathSet; + pathSet = traverser.templatePaths(sources, targets, steps, + request.capacity, request.limit); + + if (!request.withVertex) { + return manager.serializer(g).writePaths("paths", pathSet, false); + } + + Set ids = new HashSet<>(); + for (HugeTraverser.Path p : pathSet) { + ids.addAll(p.vertices()); + } + Iterator iter = QueryResults.emptyIterator(); + if (!ids.isEmpty()) { + iter = g.vertices(ids.toArray()); + } + return manager.serializer(g).writePaths("paths", pathSet, + false, iter); + } + + private static List steps(HugeGraph g, List steps) { + List edgeSteps = new ArrayList<>(steps.size()); + for (TraverserAPI.Step step : steps) { + edgeSteps.add(step(g, step)); + } + return edgeSteps; + } + + private static class Request { + + @JsonProperty("sources") + public Vertices sources; + @JsonProperty("targets") + public Vertices targets; + @JsonProperty("steps") + public List steps; + @JsonProperty("capacity") + public long capacity = Long.valueOf(DEFAULT_CAPACITY); + @JsonProperty("limit") + public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); + @JsonProperty("with_vertex") + public boolean withVertex = false; + + @Override + public String toString() { + return String.format("PathRequest{sources=%s,targets=%s,steps=%s," + + "capacity=%s,limit=%s,withVertex=%s}", + this.sources, this.targets, this.steps, + this.capacity, this.limit, this.withVertex); + } + } +} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java new file mode 100644 index 0000000000..c1fc086660 --- /dev/null +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java @@ -0,0 +1,61 @@ +/* + * 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.api.traversers; + +import java.util.List; +import java.util.Map; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.api.API; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.type.define.Directions; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; + +public class TraverserAPI extends API { + + protected static EdgeStep step(HugeGraph graph, Step step) { + return new EdgeStep(graph, step.direction, step.labels, step.properties, + step.degree, step.skipDegree); + } + + protected static class Step { + + @JsonProperty("direction") + public Directions direction; + @JsonProperty("labels") + public List labels; + @JsonProperty("properties") + public Map properties; + @JsonProperty("degree") + public long degree = Long.valueOf(DEFAULT_DEGREE); + @JsonProperty("skip_degree") + public long skipDegree = 0L; + + @Override + public String toString() { + return String.format("Step{direction=%s,labels=%s,properties=%s," + + "degree=%s,skipDegree=%s}", + this.direction, this.labels, this.properties, + this.degree, this.skipDegree); + } + } +} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/SourceVertices.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/Vertices.java similarity index 97% rename from hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/SourceVertices.java rename to hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/Vertices.java index 65b24d48b4..4c0a0f5685 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/SourceVertices.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/Vertices.java @@ -37,7 +37,7 @@ import com.baidu.hugegraph.util.E; import com.fasterxml.jackson.annotation.JsonProperty; -public class SourceVertices { +public class Vertices { @JsonProperty("ids") public Set ids; @@ -46,7 +46,7 @@ public class SourceVertices { @JsonProperty("properties") public Map properties; - public Iterator sourcesVertices(HugeGraph g) { + public Iterator vertices(HugeGraph g) { Map props = this.properties; E.checkArgument(!((this.ids == null || this.ids.isEmpty()) && (props == null || props.isEmpty()) && diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java index 3d19d09f07..2b09b9f915 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java @@ -25,6 +25,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal; import org.apache.tinkerpop.gremlin.structure.Edge; @@ -50,6 +51,7 @@ import com.baidu.hugegraph.util.JsonUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; public class JsonSerializer implements Serializer { @@ -286,4 +288,23 @@ public String writeWeightedPaths(WeightedPaths paths, return JsonUtil.toJson(ImmutableMap.of("paths", paths.toMap(), "vertices", vertices)); } + + @Override + public String writeNodesWithPath(String name, Set nodes, + Collection paths, + Iterator iterator, + boolean countOnly) { + List> pathList; + pathList = new ArrayList<>(); + for (HugeTraverser.Path path : paths) { + pathList.add(path.toMap(false)); + } + + Map results; + results = ImmutableMap.of("size", nodes.size(), + name, countOnly ? ImmutableSet.of() : nodes, + "paths", pathList, + "vertices", iterator); + return JsonUtil.toJson(results); + } } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/Serializer.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/Serializer.java index d04849655e..8c680f113b 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/Serializer.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/Serializer.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Vertex; @@ -34,6 +35,7 @@ import com.baidu.hugegraph.schema.PropertyKey; import com.baidu.hugegraph.schema.VertexLabel; import com.baidu.hugegraph.traversal.algorithm.CustomizedCrosspointsTraverser.CrosspointsPaths; + import com.baidu.hugegraph.traversal.algorithm.FusiformSimilarityTraverser.SimilarsMap; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.traversal.algorithm.SingleSourceShortestPathTraverser.NodeWithWeight; @@ -98,4 +100,9 @@ public String writeWeightedPath(NodeWithWeight path, public String writeWeightedPaths(WeightedPaths paths, Iterator vertices); + + public String writeNodesWithPath(String name, Set nodes, + Collection paths, + Iterator iterator, + boolean countOnly); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java index a89fb3c255..82eb0c305b 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java @@ -503,4 +503,12 @@ public static synchronized CoreOptions instance() { disallowEmpty(), "./conf/computer.yaml" ); + + public static final ConfigOption OLTP_CONCURRENT_THREADS = + new ConfigOption<>( + "oltp.concurrent.threads", + "Thread number to concurrently execute oltp algorithm.", + rangeInt(0, 200), + 10 + ); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java new file mode 100644 index 0000000000..ddbef3d5fb --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java @@ -0,0 +1,113 @@ +/* + * 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.traversal.algorithm; + +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.CollectionUtil; +import com.baidu.hugegraph.util.E; + +public class CustomizedKneighborTraverser extends TpTraverser { + + public CustomizedKneighborTraverser(HugeGraph graph) { + super(graph, "kneighbor"); + } + + public Set customizedKneighbor(Id source, EdgeStep step, + int maxDepth, long limit) { + E.checkNotNull(source, "source vertex id"); + this.checkVertexExist(source, "source vertex"); + checkPositive(maxDepth, "k-neighbor max_depth"); + checkLimit(limit); + + if (step.direction == Directions.BOTH && maxDepth > 3) { + return this.customizedKneighborConcurrent(source, step, + maxDepth, limit); + } else { + return this.customizedKneighborSingle(source, step, + maxDepth, limit); + } + } + + public Set customizedKneighborConcurrent(Id source, EdgeStep step, + int maxDepth, long limit) { + Set latest = ConcurrentHashMap.newKeySet(); + Set all = ConcurrentHashMap.newKeySet(); + + Node sourceV = new KNode(source, null); + + latest.add(sourceV); + all.add(sourceV); + + while (maxDepth-- > 0) { + long remaining = limit == NO_LIMIT ? NO_LIMIT : limit - all.size(); + AtomicLong remain = new AtomicLong(remaining); + latest = this.adjacentVertices(latest, step, all, remain); + int size = all.size() + latest.size(); + if (limit != NO_LIMIT && size >= limit) { + int subLength = (int) limit - all.size(); + Iterator iterator = latest.iterator(); + for (int i = 0; i < subLength && iterator.hasNext(); i++) { + all.add(iterator.next()); + } + break; + } else { + all.addAll(latest); + } + } + + return all; + } + + public Set customizedKneighborSingle(Id source, EdgeStep step, + int maxDepth, long limit) { + Set latest = newSet(); + Set all = newSet(); + + Node sourceV = new KNode(source, null); + + latest.add(sourceV); + all.add(sourceV); + + while (maxDepth-- > 0) { + long remaining = limit == NO_LIMIT ? NO_LIMIT : limit - all.size(); + latest = this.adjacentVertices(latest, step, all, remaining); + int size = all.size() + latest.size(); + if (limit != NO_LIMIT && size >= limit) { + int subLength = (int) limit - all.size(); + Iterator iterator = latest.iterator(); + for (int i = 0; i < subLength && iterator.hasNext(); i++) { + all.add(iterator.next()); + } + break; + } else { + all.addAll(latest); + } + } + + return all; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java new file mode 100644 index 0000000000..6fb9a14e96 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java @@ -0,0 +1,135 @@ +/* + * 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.traversal.algorithm; + +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import com.baidu.hugegraph.HugeException; +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.CollectionUtil; +import com.baidu.hugegraph.util.E; + +public class CustomizedKoutTraverser extends TpTraverser { + + public CustomizedKoutTraverser(HugeGraph graph) { + super(graph, "kout"); + } + + public Set customizedKout(Id source, EdgeStep step, int maxDepth, + boolean nearest, long capacity, + long limit) { + E.checkNotNull(source, "source vertex id"); + this.checkVertexExist(source, "source vertex"); + checkPositive(maxDepth, "k-out max_depth"); + checkCapacity(capacity); + checkLimit(limit); + + Set results = newSet(); + if (step.direction == Directions.BOTH && maxDepth > 3) { + results = this.customizedKoutConcurrent(source, step, maxDepth, + nearest, capacity); + } else { + results = this.customizedKoutSingle(source, step, maxDepth, + nearest, capacity); + } + + if (limit != NO_LIMIT && results.size() > limit) { + results = CollectionUtil.subSet(results, 0, (int) limit); + } + + return results; + } + + public Set customizedKoutConcurrent(Id source, EdgeStep step, + int maxDepth, boolean nearest, + long capacity) { + Set latest = ConcurrentHashMap.newKeySet(); + Set all = ConcurrentHashMap.newKeySet(); + + Node sourceV = new KNode(source, null); + + latest.add(sourceV); + all.add(sourceV); + + int depth = maxDepth; + long remaining = capacity == NO_LIMIT ? + NO_LIMIT : capacity - latest.size(); + while (depth-- > 0) { + AtomicLong remain = new AtomicLong(remaining); + if (nearest) { + latest = this.adjacentVertices(latest, step, all, remain); + all.addAll(latest); + } else { + latest = this.adjacentVertices(latest, step, null, remain); + } + if (capacity != NO_LIMIT) { + // Update 'remaining' value to record remaining capacity + remaining -= latest.size(); + reachCapacity(remaining, capacity, depth); + } + } + + return latest; + } + + public Set customizedKoutSingle(Id source, EdgeStep step, + int maxDepth, boolean nearest, + long capacity) { + Set latest = newSet(); + Set all = newSet(); + + Node sourceV = new KNode(source, null); + + latest.add(sourceV); + all.add(sourceV); + + int depth = maxDepth; + long remaining = capacity == NO_LIMIT ? + NO_LIMIT : capacity - latest.size(); + while (depth-- > 0) { + if (nearest) { + latest = this.adjacentVertices(latest, step, all, remaining); + all.addAll(latest); + } else { + latest = this.adjacentVertices(latest, step, null, remaining); + } + if (capacity != NO_LIMIT) { + // Update 'remaining' value to record remaining capacity + remaining -= latest.size(); + reachCapacity(remaining, capacity, depth); + } + } + + return latest; + } + + private static void reachCapacity(long remaining, long capacity, + int depth) { + if (remaining <= 0 && depth > 0) { + throw new HugeException( + "Reach capacity '%s' while remaining depth '%s'", + capacity, depth); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/EdgeStep.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/EdgeStep.java index f9a43b6d88..f27e435b84 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/EdgeStep.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/EdgeStep.java @@ -37,7 +37,7 @@ public class EdgeStep { - protected final Directions direction; + protected Directions direction; protected final Map labels; protected final Map properties; protected final long degree; @@ -80,6 +80,10 @@ public Id[] edgeLabels() { return edgeLabels; } + public void swithDirection() { + this.direction = this.direction.opposite(); + } + public long limit() { long limit = this.skipDegree > 0L ? this.skipDegree : this.degree; return limit; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java index 0cc21abf50..9bc8f65d70 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java @@ -46,6 +46,7 @@ import com.baidu.hugegraph.backend.tx.GraphTransaction; import com.baidu.hugegraph.exception.NotFoundException; import com.baidu.hugegraph.iterator.ExtendableIterator; +import com.baidu.hugegraph.iterator.FilterIterator; import com.baidu.hugegraph.iterator.MapperIterator; import com.baidu.hugegraph.schema.SchemaLabel; import com.baidu.hugegraph.structure.HugeEdge; @@ -76,6 +77,8 @@ public class HugeTraverser { public static final String DEFAULT_MAX_DEPTH = "50"; public static final String DEFAULT_WEIGHT = "0"; + protected static final int MAX_VERTICES = 10; + // Empirical value of scan limit, with which results can be returned in 3s public static final String DEFAULT_PAGE_LIMIT = "100000"; @@ -217,10 +220,12 @@ public double jaccardSimilarity(Id vertex, Id other, Directions dir, vertex, dir, labelId, degree)); Set targetNeighbors = IteratorUtils.set(this.adjacentVertices( other, dir, labelId, degree)); - int interNum = CollectionUtil.intersect(sourceNeighbors, - targetNeighbors).size(); - int unionNum = CollectionUtil.union(sourceNeighbors, - targetNeighbors).size(); + return jaccardSimilarity(sourceNeighbors, targetNeighbors); + } + + public double jaccardSimilarity(Set set1, Set set2) { + int interNum = CollectionUtil.intersect(set1, set2).size(); + int unionNum = CollectionUtil.union(set1, set2).size(); return (double) interNum / unionNum; } @@ -259,6 +264,35 @@ protected Iterator adjacentVertices(Id source, Directions dir, }); } + protected Set adjacentVertices(Set vertices, EdgeStep step, + Set excluded, long remaining) { + Set neighbors = newSet(); + for (Node source : vertices) { + Iterator edges = this.edgesOfVertex(source.id(), step); + while (edges.hasNext()) { + Id target = ((HugeEdge) edges.next()).id().otherVertexId(); + KNode kNode = new KNode(target, (KNode) source); + if (excluded != null && excluded.contains(kNode)) { + continue; + } + neighbors.add(kNode); + if (--remaining <= 0L) { + return neighbors; + } + } + } + return neighbors; + } + + protected Set adjacentVertices(Id source, EdgeStep step) { + Set neighbors = new HashSet<>(); + Iterator edges = this.edgesOfVertex(source, step); + while (edges.hasNext()) { + neighbors.add(((HugeEdge) edges.next()).id().otherVertexId()); + } + return neighbors; + } + protected Iterator edgesOfVertex(Id source, Directions dir, Id label, long limit) { Id[] labels = {}; @@ -309,16 +343,24 @@ private Iterator edgesOfVertex(Id source, EdgeStep edgeStep, Query query = GraphTransaction.constructEdgesQuery(source, edgeStep.direction, edgeLabels); + ConditionQuery filter = null; if (mustAllSK) { this.fillFilterBySortKeys(query, edgeLabels, edgeStep.properties); } else { - this.fillFilterByProperties(query, edgeStep.properties); + filter = (ConditionQuery) query.copy(); + this.fillFilterByProperties(filter, edgeStep.properties); } query.capacity(Query.NO_CAPACITY); if (edgeStep.limit() != NO_LIMIT) { query.limit(edgeStep.limit()); } Iterator edges = this.graph().edges(query); + if (filter != null) { + ConditionQuery finalFilter = filter; + edges = new FilterIterator<>(edges, (e) -> { + return finalFilter.test((HugeEdge) e); + }); + } return edgeStep.skipSuperNodeIfNeeded(edges); } @@ -590,6 +632,22 @@ public boolean equals(Object object) { } } + public static class KNode extends Node { + + public KNode(Id id, KNode parent) { + super(id, parent); + } + + @Override + public boolean equals(Object object) { + if (!(object instanceof KNode)) { + return false; + } + KNode other = (KNode) object; + return Objects.equals(this.id(), other.id()); + } + } + public static class Path { public static final Path EMPTY_PATH = new Path(ImmutableList.of()); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java new file mode 100644 index 0000000000..20d4dcb582 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java @@ -0,0 +1,182 @@ +/* + * 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.traversal.algorithm; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import com.baidu.hugegraph.HugeException; +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.E; +import com.google.common.collect.ImmutableMap; + +public class JaccardSimilarTraverser extends TpTraverser { + + public JaccardSimilarTraverser(HugeGraph graph) { + super(graph, "jaccard-similar"); + } + + public Map jaccardSimilars(Id source, EdgeStep step, + int top, long capacity) { + E.checkNotNull(source, "source vertex id"); + this.checkVertexExist(source, "source vertex"); + checkCapacity(capacity); + + Map results; + if (step.direction == Directions.BOTH) { + results = this.jaccardSimilarsConcurrent(source, step, capacity); + } else { + results = this.jaccardSimilarsSingle(source, step, capacity); + } + + if (top > 0) { + results = HugeTraverser.topN(results, true, top); + } + + return results; + } + + public Map jaccardSimilarsConcurrent(Id source, EdgeStep step, + long capacity) { + AtomicLong count = new AtomicLong(0L); + Set accessed = ConcurrentHashMap.newKeySet(); + accessed.add(source); + reachCapacity(count.incrementAndGet(), capacity); + + // Query neighbors + Set layer1s = this.adjacentVertices(source, step); + reachCapacity(count.get() + layer1s.size(), capacity); + count.addAndGet(layer1s.size()); + if (layer1s.isEmpty()) { + return ImmutableMap.of(); + } + + Map results = new ConcurrentHashMap<>(); + Set layer2All = ConcurrentHashMap.newKeySet(); + + this.traverseIds(layer1s.iterator(), id -> { + // Skip if accessed already + if (accessed.contains(id)) { + return; + } + Set layer2s = this.adjacentVertices(id, step); + if (layer2s.isEmpty()) { + results.put(id, 0.0D); + } + + layer2All.addAll(layer2s); + reachCapacity(count.get() + layer2All.size(), capacity); + double jaccardSimilarity = this.jaccardSimilarity(layer1s, layer2s); + results.put(id, jaccardSimilarity); + accessed.add(id); + }); + + count.addAndGet(layer2All.size()); + + this.traverseIds(layer2All.iterator(), id -> { + // Skip if accessed already + if (accessed.contains(id)) { + return; + } + Set layer3s = this.adjacentVertices(id, step); + reachCapacity(count.get() + layer3s.size(), capacity); + if (layer3s.isEmpty()) { + results.put(id, 0.0D); + } + + double jaccardSimilarity = this.jaccardSimilarity(layer1s, layer3s); + results.put(id, jaccardSimilarity); + accessed.add(id); + }); + + return results; + } + + public Map jaccardSimilarsSingle(Id source, EdgeStep step, + long capacity) { + long count = 0L; + Set accessed = new HashSet<>(); + accessed.add(source); + reachCapacity(++count, capacity); + + // Query neighbors + Set layer1s = this.adjacentVertices(source, step); + reachCapacity(count + layer1s.size(), capacity); + count += layer1s.size(); + if (layer1s.isEmpty()) { + return ImmutableMap.of(); + } + + Map results = new HashMap<>(); + Set layer2s; + Set layer2All = new HashSet<>(); + double jaccardSimilarity; + for (Id neighbor : layer1s) { + // Skip if accessed already + if (accessed.contains(neighbor)) { + continue; + } + layer2s = this.adjacentVertices(neighbor, step); + if (layer2s.isEmpty()) { + results.put(neighbor, 0.0D); + continue; + } + + layer2All.addAll(layer2s); + reachCapacity(count + layer2All.size(), capacity); + jaccardSimilarity = this.jaccardSimilarity(layer1s, layer2s); + results.put(neighbor, jaccardSimilarity); + accessed.add(neighbor); + } + count += layer2All.size(); + + Set layer3s; + for (Id neighbor : layer2All) { + // Skip if accessed already + if (accessed.contains(neighbor)) { + continue; + } + layer3s = this.adjacentVertices(neighbor, step); + reachCapacity(count + layer3s.size(), capacity); + if (layer3s.isEmpty()) { + results.put(neighbor, 0.0D); + continue; + } + + jaccardSimilarity = this.jaccardSimilarity(layer1s, layer3s); + results.put(neighbor, jaccardSimilarity); + accessed.add(neighbor); + } + + return results; + } + + private static void reachCapacity(long count, long capacity) { + if (capacity != NO_LIMIT && count > capacity) { + throw new HugeException("Reach capacity '%s'", capacity); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java new file mode 100644 index 0000000000..b87cfd289d --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java @@ -0,0 +1,130 @@ +/* + * 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.traversal.algorithm; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Consumer; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.E; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.Path.EMPTY_PATH; + +public class MultiNodeShortestPathTraverser extends TpTraverser { + + public MultiNodeShortestPathTraverser(HugeGraph graph) { + super(graph, "multi-node-shortest-path"); + } + + public List multiNodeShortestPath(Iterator vertices, + EdgeStep step, int maxDepth, + long capacity) { + List vertexList = IteratorUtils.list(vertices); + int size = vertexList.size(); + E.checkState(size >= 2 && size <= MAX_VERTICES, + "The number of vertices of multiple node shortest path " + + "must in [2, %s], but got: %s", + MAX_VERTICES, vertexList.size()); + List> pairs = new ArrayList<>(); + cmn(vertexList, size, 2, 0, null, r -> { + Id source = ((HugeVertex) r.get(0)).id(); + Id target = ((HugeVertex) r.get(1)).id(); + Pair pair = Pair.of(source, target); + pairs.add(pair); + }); + + if (step.direction == Directions.BOTH && maxDepth > 3 || size > 10) { + return this.multiNodeShortestPathConcurrent(pairs, step, + maxDepth, capacity); + } else { + return this.multiNodeShortestPathSingle(pairs, step, + maxDepth, capacity); + } + } + + public List multiNodeShortestPathConcurrent(List> pairs, + EdgeStep step, + int maxDepth, + long capacity) { + List results = new CopyOnWriteArrayList<>(); + ShortestPathTraverser traverser = + new ShortestPathTraverser(this.graph()); + this.traversePairs(pairs.iterator(), pair -> { + Path path = traverser.shortestPath(pair.getLeft(), pair.getRight(), + step, maxDepth, capacity); + if (!EMPTY_PATH.equals(path)) { + results.add(path); + } + }); + + return results; + } + + public List multiNodeShortestPathSingle(List> pairs, + EdgeStep step, int maxDepth, + long capacity) { + List results = new ArrayList<>(); + ShortestPathTraverser traverser = + new ShortestPathTraverser(this.graph()); + for (Pair pair : pairs) { + Path path = traverser.shortestPath(pair.getLeft(), pair.getRight(), + step, maxDepth, capacity); + if (!EMPTY_PATH.equals(path)) { + results.add(path); + } + } + return results; + } + + private static void cmn(List all, int m, int n, int current, + List result, Consumer> consumer) { + assert m <= all.size(); + assert current < all.size(); + if (result == null) { + result = new ArrayList<>(n); + } + if (n == 0) { + // All n items are selected + consumer.accept(result); + return; + } + if (m < n || current >= all.size()) { + return; + } + + // Select current item, continue to select C(m-1, n-1) + int index = result.size(); + result.add(all.get(current)); + cmn(all, m - 1, n - 1, ++current, result, consumer); + // Not select current item, continue to select C(m-1, n) + result.remove(index); + cmn(all, m - 1, n, current, result, consumer); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/ShortestPathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/ShortestPathTraverser.java index e72cc96139..aaec88379b 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/ShortestPathTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/ShortestPathTraverser.java @@ -19,8 +19,11 @@ package com.baidu.hugegraph.traversal.algorithm; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import org.apache.tinkerpop.gremlin.structure.Edge; @@ -40,7 +43,7 @@ public ShortestPathTraverser(HugeGraph graph) { } public Path shortestPath(Id sourceV, Id targetV, Directions dir, - String label, int depth, long degree, + List labels, int depth, long degree, long skipDegree, long capacity) { E.checkNotNull(sourceV, "source vertex id"); E.checkNotNull(targetV, "target vertex id"); @@ -56,8 +59,11 @@ public Path shortestPath(Id sourceV, Id targetV, Directions dir, return new Path(ImmutableList.of(sourceV)); } - Id labelId = this.getEdgeLabelId(label); - Traverser traverser = new Traverser(sourceV, targetV, dir, labelId, + Map labelMap = new HashMap<>(labels.size()); + for (String label : labels) { + labelMap.put(this.getEdgeLabelId(label), label); + } + Traverser traverser = new Traverser(sourceV, targetV, dir, labelMap, degree, skipDegree, capacity); PathSet paths; while (true) { @@ -81,8 +87,15 @@ public Path shortestPath(Id sourceV, Id targetV, Directions dir, return paths.isEmpty() ? Path.EMPTY_PATH : paths.iterator().next(); } + public Path shortestPath(Id sourceV, Id targetV, EdgeStep step, + int depth, long capacity) { + return this.shortestPath(sourceV, targetV, step.direction, + new ArrayList<>(step.labels.values()), + depth, step.degree, step.skipDegree, capacity); + } + public PathSet allShortestPaths(Id sourceV, Id targetV, Directions dir, - String label, int depth, long degree, + List labels, int depth, long degree, long skipDegree, long capacity) { E.checkNotNull(sourceV, "source vertex id"); E.checkNotNull(targetV, "target vertex id"); @@ -100,8 +113,11 @@ public PathSet allShortestPaths(Id sourceV, Id targetV, Directions dir, return paths; } - Id labelId = this.getEdgeLabelId(label); - Traverser traverser = new Traverser(sourceV, targetV, dir, labelId, + Map labelMap = new HashMap<>(labels.size()); + for (String label : labels) { + labelMap.put(this.getEdgeLabelId(label), label); + } + Traverser traverser = new Traverser(sourceV, targetV, dir, labelMap, degree, skipDegree, capacity); while (true) { // Found, reach max depth or reach capacity, stop searching @@ -130,18 +146,19 @@ private class Traverser { private Map targets = newMap(); private final Directions direction; - private final Id label; + private final Map labels; private final long degree; private final long skipDegree; private final long capacity; private long size; - public Traverser(Id sourceV, Id targetV, Directions dir, Id label, - long degree, long skipDegree, long capacity) { + public Traverser(Id sourceV, Id targetV, Directions dir, + Map labels, long degree, + long skipDegree, long capacity) { this.sources.put(sourceV, new Node(sourceV)); this.targets.put(targetV, new Node(targetV)); this.direction = dir; - this.label = label; + this.labels = labels; this.degree = degree; this.skipDegree = skipDegree; this.capacity = capacity; @@ -158,7 +175,7 @@ public PathSet forward(boolean all) { // Traversal vertices of previous level for (Node v : this.sources.values()) { Iterator edges = edgesOfVertex(v.id(), this.direction, - this.label, degree); + this.labels, degree); edges = skipSuperNodeIfNeeded(edges, this.degree, this.skipDegree); while (edges.hasNext()) { @@ -209,7 +226,7 @@ public PathSet backward(boolean all) { // Traversal vertices of previous level for (Node v : this.targets.values()) { Iterator edges = edgesOfVertex(v.id(), opposite, - this.label, degree); + this.labels, degree); edges = skipSuperNodeIfNeeded(edges, this.degree, this.skipDegree); while (edges.hasNext()) { @@ -254,7 +271,7 @@ private boolean superNode(Id vertex, Directions direction) { return false; } Iterator edges = edgesOfVertex(vertex, direction, - this.label, this.skipDegree); + this.labels, this.skipDegree); return IteratorUtils.count(edges) >= this.skipDegree; } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java new file mode 100644 index 0000000000..34a021ee29 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -0,0 +1,418 @@ +/* + * 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.traversal.algorithm; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import javax.ws.rs.core.MultivaluedMap; + +import org.apache.tinkerpop.gremlin.structure.Edge; +import org.apache.tinkerpop.gremlin.structure.Vertex; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.util.E; + +public class TemplatePathsTraverser extends TpTraverser { + + public TemplatePathsTraverser(HugeGraph graph) { + super(graph, "template-paths"); + } + + @SuppressWarnings("unchecked") + public PathSet templatePaths(Iterator sources, + Iterator targets, + List steps, + long capacity, long limit) { + checkCapacity(capacity); + checkLimit(limit); + + List sourceList = new ArrayList<>(); + while (sources.hasNext()) { + sourceList.add(((HugeVertex) sources.next()).id()); + } + int sourceSize = sourceList.size(); + E.checkState(sourceSize >= 1 && sourceSize <= MAX_VERTICES, + "The number of source vertices must in [1, %s], " + + "but got: %s", MAX_VERTICES, sourceList.size()); + List targetList = new ArrayList<>(); + while (targets.hasNext()) { + targetList.add(((HugeVertex) targets.next()).id()); + } + int targetSize = targetList.size(); + E.checkState(targetSize >= 1 && targetSize <= MAX_VERTICES, + "The number of target vertices must in [1, %s], " + + "but got: %s", MAX_VERTICES, sourceList.size()); + + Traverser traverser = steps.size() > 4 ? + new ConcurrentTraverser(sourceList, targetList, + steps, capacity, limit) : + new SingleTraverser(sourceList, targetList, + steps, capacity, limit); + PathSet paths; + do { + // Forward + paths = traverser.forward(); + if (traverser.finish()) { + return paths; + } + + // Backward + paths = traverser.backward(); + if (traverser.finish()) { + for (Path path : paths) { + path.reverse(); + } + return paths; + } + } while (true); + } + + private class Traverser { + + protected final List steps; + protected int stepCount; + protected final long capacity; + protected final long limit; + + protected long pathCount; + + public Traverser(List steps, long capacity, long limit) { + this.steps = steps; + this.capacity = capacity; + this.limit = limit; + + this.pathCount = 0L; + this.stepCount = 0; + } + + public PathSet forward() { + return new PathSet(); + } + + public PathSet backward() { + return new PathSet(); + } + + protected boolean finish() { + return this.stepCount == this.steps.size(); + } + + protected boolean lastStep() { + return this.stepCount == this.steps.size() - 1; + } + + protected int accessedNodes() { + return 0; + } + + protected boolean reachLimit() { + checkCapacity(this.capacity, this.accessedNodes(), + "template paths"); + if (this.limit == NO_LIMIT || this.pathCount < this.limit) { + return false; + } + return true; + } + } + + private class ConcurrentTraverser extends Traverser { + + private ConcurrentMultiValuedMap sources = + new ConcurrentMultiValuedMap<>(); + private ConcurrentMultiValuedMap targets = + new ConcurrentMultiValuedMap<>(); + + public ConcurrentTraverser(Collection sources, + Collection targets, List steps, + long capacity, long limit) { + super(steps, capacity, limit); + for (Id id : sources) { + this.sources.add(id, new Node(id)); + } + for (Id id : targets) { + this.targets.add(id, new Node(id)); + } + } + + /** + * Search forward from sources + */ + public PathSet forward() { + PathSet paths = new PathSet(); + ConcurrentMultiValuedMap newVertices = + new ConcurrentMultiValuedMap<>(); + EdgeStep step = this.steps.get(this.stepCount / 2); + // Traversal vertices of previous level + traverseIds(this.sources.keySet().iterator(), vid -> { + if (this.reachLimit()) { + return; + } + Iterator edges = edgesOfVertex(vid, step); + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : this.sources.get(vid)) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.lastStep() && + this.targets.containsKey(target)) { + for (Node node : this.targets.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + }); + + // Re-init sources + this.sources = newVertices; + + this.stepCount++; + + return paths; + } + + /** + * Search backward from target + */ + public PathSet backward() { + PathSet paths = new PathSet(); + ConcurrentMultiValuedMap newVertices = + new ConcurrentMultiValuedMap<>(); + int index = this.steps.size() - stepCount / 2 - 1; + EdgeStep step = this.steps.get(index); + step.swithDirection(); + // Traversal vertices of previous level + traverseIds(this.targets.keySet().iterator(), vid -> { + if (this.reachLimit()) { + return; + } + Iterator edges = edgesOfVertex(vid, step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : this.targets.get(vid)) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.lastStep() && + this.sources.containsKey(target)) { + for (Node node : this.sources.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + }); + + // Re-init targets + this.targets = newVertices; + + this.stepCount++; + + return paths; + } + + protected int accessedNodes() { + return this.sources.size() + this.targets.size(); + } + + public class ConcurrentMultiValuedMap + extends ConcurrentHashMap> { + + public ConcurrentMultiValuedMap() { + super(); + } + + public void add(K key, V value) { + if (!this.contains(key)) { + this.putIfAbsent(key, ConcurrentHashMap.newKeySet()); + } + this.get(key).add(value); + } + } + } + + private class SingleTraverser extends Traverser { + + private MultivaluedMap sources = newMultivalueMap(); + private MultivaluedMap targets = newMultivalueMap(); + + public SingleTraverser(Collection sources, Collection targets, + List steps, long capacity, + long limit) { + super(steps, capacity, limit); + for (Id id : sources) { + this.sources.add(id, new Node(id)); + } + for (Id id : targets) { + this.targets.add(id, new Node(id)); + } + } + + /** + * Search forward from sources + */ + public PathSet forward() { + PathSet paths = new PathSet(); + MultivaluedMap newVertices = newMultivalueMap(); + EdgeStep step = this.steps.get(this.stepCount / 2); + Iterator edges; + // Traversal vertices of previous level + for (Map.Entry> entry : this.sources.entrySet()) { + Id vid = entry.getKey(); + edges = edgesOfVertex(vid, step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : entry.getValue()) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.lastStep() && + this.targets.containsKey(target)) { + for (Node node : this.targets.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return paths; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + } + // Re-init sources + this.sources = newVertices; + + this.stepCount++; + + return paths; + } + + /** + * Search backward from target + */ + public PathSet backward() { + PathSet paths = new PathSet(); + MultivaluedMap newVertices = newMultivalueMap(); + int index = this.steps.size() - stepCount / 2 - 1; + EdgeStep step = this.steps.get(index); + step.swithDirection(); + Iterator edges; + // Traversal vertices of previous level + for (Map.Entry> entry : this.targets.entrySet()) { + Id vid = entry.getKey(); + edges = edgesOfVertex(vid, step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : entry.getValue()) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.lastStep() && + this.sources.containsKey(target)) { + for (Node node : this.sources.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return paths; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + } + + // Re-init targets + this.targets = newVertices; + + this.stepCount++; + + return paths; + } + + protected int accessedNodes() { + return this.sources.size() + this.targets.size(); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java new file mode 100644 index 0000000000..78d62a2c06 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.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.traversal.algorithm; + +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.tinkerpop.gremlin.structure.Edge; +import org.apache.tinkerpop.gremlin.structure.Element; +import org.apache.tinkerpop.gremlin.structure.Property; +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.apache.tinkerpop.gremlin.structure.util.CloseableIterator; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.config.CoreOptions; +import com.baidu.hugegraph.config.HugeConfig; +import com.baidu.hugegraph.iterator.FilterIterator; +import com.baidu.hugegraph.job.algorithm.Consumers; +import com.baidu.hugegraph.structure.HugeEdge; + +import jersey.repackaged.com.google.common.base.Objects; + +public abstract class TpTraverser extends HugeTraverser + implements AutoCloseable { + + protected final ExecutorService executor; + + protected TpTraverser(HugeGraph graph) { + super(graph); + this.executor = null; + } + + protected TpTraverser(HugeGraph graph, String name) { + super(graph); + int workers = ((HugeConfig) graph.configuration()) + .get(CoreOptions.OLTP_CONCURRENT_THREADS); + this.executor = Consumers.newThreadPool(name, workers); + } + + @Override + public void close() { + if (this.executor != null) { + this.executor.shutdown(); + } + } + + protected Set adjacentVertices(Set vertices, EdgeStep step, + Set excluded, + AtomicLong remaining) { + Set neighbors = ConcurrentHashMap.newKeySet(); + this.traverseNodes(vertices.iterator(), v -> { + Iterator edges = this.edgesOfVertex(v.id(), step); + while (edges.hasNext()) { + Id target = ((HugeEdge) edges.next()).id().otherVertexId(); + KNode kNode = new KNode(target, (KNode) v); + if (excluded != null && excluded.contains(kNode)) { + continue; + } + neighbors.add(kNode); + if (remaining.decrementAndGet() <= 0L) { + return; + } + } + }); + return neighbors; + } + + protected long traverseNodes(Iterator vertices, + Consumer consumer) { + return this.traverse(vertices, consumer, "traverse-nodes"); + } + + protected long traversePairs(Iterator> pairs, + Consumer> consumer) { + return this.traverse(pairs, consumer, "traverse-pairs"); + } + + protected long traverseIds(Iterator ids, Consumer consumer) { + return this.traverse(ids, consumer, "traverse-ids"); + } + + protected long traverse(Iterator iterator, Consumer consumer, + String name) { + Consumers consumers = new Consumers<>(this.executor, + consumer, null); + consumers.start(name); + long total = 0L; + try { + while (iterator.hasNext()) { + total++; + K v = iterator.next(); + consumers.provide(v); + } + } catch (Consumers.StopExecution e) { + // pass + } catch (Throwable e) { + throw Consumers.wrapException(e); + } finally { + consumers.await(); + CloseableIterator.closeIterator(iterator); + } + return total; + } + + protected Iterator filter(Iterator vertices, + String key, Object value) { + return new FilterIterator<>(vertices, vertex -> { + return match(vertex, key, value); + }); + } + + protected boolean match(Element elem, String key, Object value) { + // check property key exists + this.graph().propertyKey(key); + // return true if property value exists & equals to specified value + Property p = elem.property(key); + return p.isPresent() && Objects.equal(p.value(), value); + } +} From 5660ca98fd1017fbc9e8143532f6eef3564bda38 Mon Sep 17 00:00:00 2001 From: zhoney Date: Sun, 6 Sep 2020 23:16:46 +0800 Subject: [PATCH 02/20] Szzq paths improve (#51) * template path use concurrent hashset to save paths in concurrent mode * support property filter for paths api Change-Id: I2e589f58cdfc48b4b8d16b7780b78cc17ab107d6 --- .../hugegraph/api/traversers/PathsAPI.java | 93 +++- .../api/traversers/TemplatePathAPI.java | 13 +- .../algorithm/CollectionPathsTraverser.java | 418 ++++++++++++++++++ .../traversal/algorithm/HugeTraverser.java | 2 - .../algorithm/TemplatePathsTraverser.java | 67 +-- .../traversal/algorithm/TpTraverser.java | 20 + 6 files changed, 570 insertions(+), 43 deletions(-) create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java index e54367fcee..d679193b0e 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java @@ -19,40 +19,102 @@ package com.baidu.hugegraph.api.traversers; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; + import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_PATHS_LIMIT; import javax.inject.Singleton; +import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.Context; +import org.apache.tinkerpop.gremlin.structure.Vertex; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.api.API; import com.baidu.hugegraph.api.graph.EdgeAPI; import com.baidu.hugegraph.api.graph.VertexAPI; import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.query.QueryResults; import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.traversal.algorithm.CollectionPathsTraverser; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.traversal.algorithm.PathsTraverser; import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; @Path("graphs/{graph}/traversers/paths") @Singleton -public class PathsAPI extends API { +public class PathsAPI extends TraverserAPI { private static final Logger LOG = Log.logger(RestServer.class); + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.sources, + "The sources of request can't be null"); + E.checkArgumentNotNull(request.targets, + "The targets of request can't be null"); + E.checkArgumentNotNull(request.step, + "The step of request can't be null"); + E.checkArgument(request.depth > 0, + "The depth of request must be > 0, but got: %s", + request.depth); + + LOG.debug("Graph [{}] get paths from source vertices '{}', target " + + "vertices '{}', with step '{}', max depth '{}', " + + "capacity '{}', limit '{}' and with_vertex '{}'", + graph, request.sources, request.targets, request.step, + request.depth, request.capacity, request.limit, + request.withVertex); + + HugeGraph g = graph(manager, graph); + Iterator sources = request.sources.vertices(g); + Iterator targets = request.targets.vertices(g); + EdgeStep step = step(g, request.step); + + CollectionPathsTraverser traverser = new CollectionPathsTraverser(g); + Collection paths; + paths = traverser.paths(sources, targets, step, request.depth, + request.capacity, request.limit); + + if (!request.withVertex) { + return manager.serializer(g).writePaths("paths", paths, false); + } + + Set ids = new HashSet<>(); + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } + Iterator iter = QueryResults.emptyIterator(); + if (!ids.isEmpty()) { + iter = g.vertices(ids.toArray()); + } + return manager.serializer(g).writePaths("paths", paths, false, iter); + } + @GET @Timed @Produces(APPLICATION_JSON_WITH_CHARSET) @@ -87,4 +149,31 @@ public String get(@Context GraphManager manager, limit); return manager.serializer(g).writePaths("paths", paths, false); } + + private static class Request { + + @JsonProperty("sources") + public Vertices sources; + @JsonProperty("targets") + public Vertices targets; + @JsonProperty("step") + public TraverserAPI.Step step; + @JsonProperty("max_depth") + public int depth; + @JsonProperty("capacity") + public long capacity = Long.valueOf(DEFAULT_CAPACITY); + @JsonProperty("limit") + public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); + @JsonProperty("with_vertex") + public boolean withVertex = false; + + @Override + public String toString() { + return String.format("PathRequest{sources=%s,targets=%s,step=%s," + + "maxDepth=%s,capacity=%s,limit=%s," + + "withVertex=%s}", this.sources, this.targets, + this.step, this.depth, this.capacity, + this.limit, this.withVertex); + } + } } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java index f53d34cfe5..565b19f02c 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java @@ -85,24 +85,23 @@ public String post(@Context GraphManager manager, List steps = steps(g, request.steps); TemplatePathsTraverser traverser = new TemplatePathsTraverser(g); - HugeTraverser.PathSet pathSet; - pathSet = traverser.templatePaths(sources, targets, steps, - request.capacity, request.limit); + Set paths; + paths = traverser.templatePaths(sources, targets, steps, + request.capacity, request.limit); if (!request.withVertex) { - return manager.serializer(g).writePaths("paths", pathSet, false); + return manager.serializer(g).writePaths("paths", paths, false); } Set ids = new HashSet<>(); - for (HugeTraverser.Path p : pathSet) { + for (HugeTraverser.Path p : paths) { ids.addAll(p.vertices()); } Iterator iter = QueryResults.emptyIterator(); if (!ids.isEmpty()) { iter = g.vertices(ids.toArray()); } - return manager.serializer(g).writePaths("paths", pathSet, - false, iter); + return manager.serializer(g).writePaths("paths", paths, false, iter); } private static List steps(HugeGraph g, List steps) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java new file mode 100644 index 0000000000..04587c6349 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -0,0 +1,418 @@ +/* + * 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.traversal.algorithm; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.ws.rs.core.MultivaluedMap; + +import org.apache.tinkerpop.gremlin.structure.Edge; +import org.apache.tinkerpop.gremlin.structure.Vertex; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.util.E; +import com.google.common.collect.ImmutableList; + +public class CollectionPathsTraverser extends TpTraverser { + + public CollectionPathsTraverser(HugeGraph graph) { + super(graph, "collection-paths"); + } + + @SuppressWarnings("unchecked") + public Collection paths(Iterator sources, + Iterator targets, + EdgeStep step, int depth, + long capacity, long limit) { + checkCapacity(capacity); + checkLimit(limit); + + List sourceList = new ArrayList<>(); + while (sources.hasNext()) { + sourceList.add(((HugeVertex) sources.next()).id()); + } + int sourceSize = sourceList.size(); + E.checkState(sourceSize >= 1 && sourceSize <= MAX_VERTICES, + "The number of source vertices must in [1, %s], " + + "but got: %s", MAX_VERTICES, sourceList.size()); + List targetList = new ArrayList<>(); + while (targets.hasNext()) { + targetList.add(((HugeVertex) targets.next()).id()); + } + int targetSize = targetList.size(); + E.checkState(targetSize >= 1 && targetSize <= MAX_VERTICES, + "The number of target vertices must in [1, %s], " + + "but got: %s", MAX_VERTICES, sourceList.size()); + checkPositive(depth, "max depth"); + + Traverser traverser = depth > 10 ? + new ConcurrentTraverser(sourceList, targetList, + step, capacity, limit) : + new SingleTraverser(sourceList, targetList, step, + capacity, limit); + + Collection paths = new HashSet<>(); + while (true) { + if (--depth < 0 || traverser.reachLimit()) { + break; + } + Collection foundPaths = traverser.forward(); + paths.addAll(foundPaths); + + if (--depth < 0 || traverser.reachLimit()) { + break; + } + foundPaths = traverser.backward(); + for (Path path : foundPaths) { + path.reverse(); + paths.add(path); + } + } + return paths; + } + + + private class Traverser { + + protected final EdgeStep step; + protected final long capacity; + protected final long limit; + + public Traverser(EdgeStep step, long capacity, long limit) { + this.step = step; + this.capacity = capacity; + this.limit = limit; + } + + public Collection forward() { + return ImmutableList.of(); + } + + public Collection backward() { + return ImmutableList.of(); + } + + public int pathCount() { + return 0; + } + + protected int accessedNodes() { + return 0; + } + + protected boolean reachLimit() { + checkCapacity(this.capacity, this.accessedNodes(), + "collection paths"); + if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { + return false; + } + return true; + } + } + + private class ConcurrentTraverser extends Traverser { + + private ConcurrentMultiValuedMap sources = + new ConcurrentMultiValuedMap<>(); + private ConcurrentMultiValuedMap targets = + new ConcurrentMultiValuedMap<>(); + private ConcurrentMultiValuedMap sourcesAll = + new ConcurrentMultiValuedMap<>(); + private ConcurrentMultiValuedMap targetsAll = + new ConcurrentMultiValuedMap<>(); + + protected AtomicInteger pathCount; + + public ConcurrentTraverser(Collection sources, + Collection targets, EdgeStep step, + long capacity, long limit) { + super(step, capacity, limit); + for (Id id : sources) { + this.sources.add(id, new Node(id)); + } + for (Id id : targets) { + this.targets.add(id, new Node(id)); + } + this.sourcesAll.putAll(this.sources); + this.targetsAll.putAll(this.targets); + this.pathCount = new AtomicInteger(0); + } + + /** + * Search forward from sources + */ + public Collection forward() { + Set paths = ConcurrentHashMap.newKeySet(); + ConcurrentMultiValuedMap newVertices = + new ConcurrentMultiValuedMap<>(); + // Traversal vertices of previous level + traverseIds(this.sources.keySet().iterator(), vid -> { + if (this.reachLimit()) { + return; + } + Iterator edges = edgesOfVertex(vid, this.step); + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : this.sources.get(vid)) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.targetsAll.containsKey(target)) { + for (Node node : this.targetsAll.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + this.pathCount.incrementAndGet(); + if (this.reachLimit()) { + return; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + }); + + // Re-init sources + this.sources = newVertices; + // Record all passed vertices + this.sourcesAll.putAll(newVertices); + + return paths; + } + + /** + * Search backward from target + */ + public Set backward() { + Set paths = ConcurrentHashMap.newKeySet(); + ConcurrentMultiValuedMap newVertices = + new ConcurrentMultiValuedMap<>(); + this.step.swithDirection(); + // Traversal vertices of previous level + traverseIds(this.targets.keySet().iterator(), vid -> { + if (this.reachLimit()) { + return; + } + Iterator edges = edgesOfVertex(vid, this.step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : this.targets.get(vid)) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.sourcesAll.containsKey(target)) { + for (Node node : this.sourcesAll.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + this.pathCount.incrementAndGet(); + if (this.reachLimit()) { + return; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + }); + this.step.swithDirection(); + + // Re-init targets + this.targets = newVertices; + // Record all passed vertices + this.targetsAll.putAll(newVertices); + + return paths; + } + + @Override + public int pathCount() { + return this.pathCount.get(); + } + + protected int accessedNodes() { + return this.sourcesAll.size() + this.targetsAll.size(); + } + } + + private class SingleTraverser extends Traverser { + + private MultivaluedMap sources = newMultivalueMap(); + private MultivaluedMap targets = newMultivalueMap(); + private MultivaluedMap sourcesAll = newMultivalueMap(); + private MultivaluedMap targetsAll = newMultivalueMap(); + + private int pathCount; + + public SingleTraverser(Collection sources, Collection targets, + EdgeStep step, long capacity, long limit) { + super(step, capacity, limit); + for (Id id : sources) { + this.sources.add(id, new Node(id)); + } + for (Id id : targets) { + this.targets.add(id, new Node(id)); + } + this.sourcesAll.putAll(this.sources); + this.targetsAll.putAll(this.targets); + this.pathCount = 0; + } + + /** + * Search forward from sources + */ + public PathSet forward() { + PathSet paths = new PathSet(); + MultivaluedMap newVertices = newMultivalueMap(); + Iterator edges; + // Traversal vertices of previous level + for (Map.Entry> entry : this.sources.entrySet()) { + Id vid = entry.getKey(); + edges = edgesOfVertex(vid, this.step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : entry.getValue()) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.targetsAll.containsKey(target)) { + for (Node node : this.targetsAll.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return paths; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + } + + // Re-init targets + this.sources = newVertices; + // Record all passed vertices + this.sourcesAll.putAll(newVertices); + + return paths; + } + + /** + * Search backward from target + */ + public PathSet backward() { + PathSet paths = new PathSet(); + MultivaluedMap newVertices = newMultivalueMap(); + this.step.swithDirection(); + Iterator edges; + // Traversal vertices of previous level + for (Map.Entry> entry : this.targets.entrySet()) { + Id vid = entry.getKey(); + edges = edgesOfVertex(vid, this.step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + for (Node n : entry.getValue()) { + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.sourcesAll.containsKey(target)) { + for (Node node : this.sourcesAll.get(target)) { + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return paths; + } + } + } + } + + // Add node to next start-nodes + newVertices.add(target, new Node(target, n)); + } + } + } + this.step.swithDirection(); + + // Re-init targets + this.targets = newVertices; + // Record all passed vertices + this.targetsAll.putAll(newVertices); + + return paths; + } + + @Override + public int pathCount() { + return this.pathCount; + } + + protected int accessedNodes() { + return this.sourcesAll.size() + this.targetsAll.size(); + } + } +} \ No newline at end of file diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java index 9bc8f65d70..7f823b0f91 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java @@ -65,8 +65,6 @@ public class HugeTraverser { private HugeGraph graph; - public static final List PATH_NONE = ImmutableList.of(); - public static final String DEFAULT_CAPACITY = "10000000"; public static final String DEFAULT_ELEMENTS_LIMIT = "10000000"; public static final String DEFAULT_PATHS_LIMIT = "10"; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index 34a021ee29..f527907f7d 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import javax.ws.rs.core.MultivaluedMap; @@ -45,10 +46,10 @@ public TemplatePathsTraverser(HugeGraph graph) { } @SuppressWarnings("unchecked") - public PathSet templatePaths(Iterator sources, - Iterator targets, - List steps, - long capacity, long limit) { + public Set templatePaths(Iterator sources, + Iterator targets, + List steps, + long capacity, long limit) { checkCapacity(capacity); checkLimit(limit); @@ -74,7 +75,7 @@ public PathSet templatePaths(Iterator sources, steps, capacity, limit) : new SingleTraverser(sourceList, targetList, steps, capacity, limit); - PathSet paths; + Set paths; do { // Forward paths = traverser.forward(); @@ -100,25 +101,26 @@ private class Traverser { protected final long capacity; protected final long limit; - protected long pathCount; - public Traverser(List steps, long capacity, long limit) { this.steps = steps; this.capacity = capacity; this.limit = limit; - this.pathCount = 0L; this.stepCount = 0; } - public PathSet forward() { + public Set forward() { return new PathSet(); } - public PathSet backward() { + public Set backward() { return new PathSet(); } + public int pathCount() { + return 0; + } + protected boolean finish() { return this.stepCount == this.steps.size(); } @@ -134,7 +136,7 @@ protected int accessedNodes() { protected boolean reachLimit() { checkCapacity(this.capacity, this.accessedNodes(), "template paths"); - if (this.limit == NO_LIMIT || this.pathCount < this.limit) { + if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { return false; } return true; @@ -148,6 +150,8 @@ private class ConcurrentTraverser extends Traverser { private ConcurrentMultiValuedMap targets = new ConcurrentMultiValuedMap<>(); + protected AtomicInteger pathCount; + public ConcurrentTraverser(Collection sources, Collection targets, List steps, long capacity, long limit) { @@ -158,13 +162,14 @@ public ConcurrentTraverser(Collection sources, for (Id id : targets) { this.targets.add(id, new Node(id)); } + this.pathCount = new AtomicInteger(0); } /** * Search forward from sources */ - public PathSet forward() { - PathSet paths = new PathSet(); + public Set forward() { + Set paths = ConcurrentHashMap.newKeySet(); ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); EdgeStep step = this.steps.get(this.stepCount / 2); @@ -191,7 +196,7 @@ public PathSet forward() { List path = n.joinPath(node); if (!path.isEmpty()) { paths.add(new Path(target, path)); - ++this.pathCount; + this.pathCount.incrementAndGet(); if (this.reachLimit()) { return; } @@ -216,8 +221,8 @@ public PathSet forward() { /** * Search backward from target */ - public PathSet backward() { - PathSet paths = new PathSet(); + public Set backward() { + Set paths = ConcurrentHashMap.newKeySet(); ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); int index = this.steps.size() - stepCount / 2 - 1; @@ -247,7 +252,7 @@ public PathSet backward() { List path = n.joinPath(node); if (!path.isEmpty()) { paths.add(new Path(target, path)); - ++this.pathCount; + this.pathCount.incrementAndGet(); if (this.reachLimit()) { return; } @@ -269,23 +274,13 @@ public PathSet backward() { return paths; } - protected int accessedNodes() { - return this.sources.size() + this.targets.size(); + @Override + public int pathCount() { + return this.pathCount.get(); } - public class ConcurrentMultiValuedMap - extends ConcurrentHashMap> { - - public ConcurrentMultiValuedMap() { - super(); - } - - public void add(K key, V value) { - if (!this.contains(key)) { - this.putIfAbsent(key, ConcurrentHashMap.newKeySet()); - } - this.get(key).add(value); - } + protected int accessedNodes() { + return this.sources.size() + this.targets.size(); } } @@ -294,6 +289,8 @@ private class SingleTraverser extends Traverser { private MultivaluedMap sources = newMultivalueMap(); private MultivaluedMap targets = newMultivalueMap(); + private int pathCount; + public SingleTraverser(Collection sources, Collection targets, List steps, long capacity, long limit) { @@ -304,6 +301,7 @@ public SingleTraverser(Collection sources, Collection targets, for (Id id : targets) { this.targets.add(id, new Node(id)); } + this.pathCount = 0; } /** @@ -411,6 +409,11 @@ public PathSet backward() { return paths; } + @Override + public int pathCount() { + return this.pathCount; + } + protected int accessedNodes() { return this.sources.size() + this.targets.size(); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index 78d62a2c06..34a52c6afd 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -139,4 +139,24 @@ protected boolean match(Element elem, String key, Object value) { Property p = elem.property(key); return p.isPresent() && Objects.equal(p.value(), value); } + + public class ConcurrentMultiValuedMap + extends ConcurrentHashMap> { + + public ConcurrentMultiValuedMap() { + super(); + } + + public void add(K key, V value) { + Set values = this.get(key); + if (values == null) { + values = ConcurrentHashMap.newKeySet(); + Set old = this.putIfAbsent(key, values); + if (old != null) { + values = old; + } + } + values.add(value); + } + } } From 851760ba26dc70bd08634e8d389293021d75f62c Mon Sep 17 00:00:00 2001 From: zhoney Date: Sun, 6 Sep 2020 23:32:45 +0800 Subject: [PATCH 03/20] fix configuration not supported with auth (#55) also paths and template paths use single thread Change-Id: I4375c455f377d9e4d43c0284ec30da1725fdf6bc --- .../hugegraph/traversal/algorithm/CollectionPathsTraverser.java | 2 +- .../hugegraph/traversal/algorithm/TemplatePathsTraverser.java | 2 +- .../com/baidu/hugegraph/traversal/algorithm/TpTraverser.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java index 04587c6349..a033b537b8 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -73,7 +73,7 @@ public Collection paths(Iterator sources, "but got: %s", MAX_VERTICES, sourceList.size()); checkPositive(depth, "max depth"); - Traverser traverser = depth > 10 ? + Traverser traverser = depth > 20 ? new ConcurrentTraverser(sourceList, targetList, step, capacity, limit) : new SingleTraverser(sourceList, targetList, step, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index f527907f7d..4cc334bd8b 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -70,7 +70,7 @@ public Set templatePaths(Iterator sources, "The number of target vertices must in [1, %s], " + "but got: %s", MAX_VERTICES, sourceList.size()); - Traverser traverser = steps.size() > 4 ? + Traverser traverser = steps.size() > 20 ? new ConcurrentTraverser(sourceList, targetList, steps, capacity, limit) : new SingleTraverser(sourceList, targetList, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index 34a52c6afd..ac9a7e744d 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -55,7 +55,7 @@ protected TpTraverser(HugeGraph graph) { protected TpTraverser(HugeGraph graph, String name) { super(graph); - int workers = ((HugeConfig) graph.configuration()) + int workers = ((HugeConfig) graph.hugegraph().configuration()) .get(CoreOptions.OLTP_CONCURRENT_THREADS); this.executor = Consumers.newThreadPool(name, workers); } From 64088e548bd4008f550d4a430153825920d79337 Mon Sep 17 00:00:00 2001 From: zhoney Date: Wed, 9 Sep 2020 16:21:42 +0800 Subject: [PATCH 04/20] template path supports repeat times args (#57) paths supports nearest args oltp multiple threads reuse add multiple thread depth config Change-Id: I8c2f921ffb56302fb8d300c09a5a13857dac187a --- .../hugegraph/api/traversers/PathsAPI.java | 12 +- .../hugegraph/api/traversers/RingsAPI.java | 2 +- .../api/traversers/TemplatePathAPI.java | 41 +++- .../api/traversers/TraverserAPI.java | 11 + .../baidu/hugegraph/config/CoreOptions.java | 13 +- .../algorithm/CollectionPathsTraverser.java | 187 +++++++++++++++- .../CustomizedKneighborTraverser.java | 4 +- .../algorithm/CustomizedKoutTraverser.java | 5 +- .../algorithm/JaccardSimilarTraverser.java | 3 +- .../MultiNodeShortestPathTraverser.java | 10 +- .../algorithm/TemplatePathsTraverser.java | 208 +++++++++++++++--- .../traversal/algorithm/TpTraverser.java | 31 ++- 12 files changed, 449 insertions(+), 78 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java index d679193b0e..16f963cd0f 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java @@ -98,7 +98,8 @@ public String post(@Context GraphManager manager, CollectionPathsTraverser traverser = new CollectionPathsTraverser(g); Collection paths; paths = traverser.paths(sources, targets, step, request.depth, - request.capacity, request.limit); + request.nearest, request.capacity, + request.limit); if (!request.withVertex) { return manager.serializer(g).writePaths("paths", paths, false); @@ -160,6 +161,8 @@ private static class Request { public TraverserAPI.Step step; @JsonProperty("max_depth") public int depth; + @JsonProperty("nearest") + public boolean nearest = true; @JsonProperty("capacity") public long capacity = Long.valueOf(DEFAULT_CAPACITY); @JsonProperty("limit") @@ -170,9 +173,10 @@ private static class Request { @Override public String toString() { return String.format("PathRequest{sources=%s,targets=%s,step=%s," + - "maxDepth=%s,capacity=%s,limit=%s," + - "withVertex=%s}", this.sources, this.targets, - this.step, this.depth, this.capacity, + "maxDepth=%s,nearest=%s,capacity=%s," + + "limit=%s,withVertex=%s}", this.sources, + this.targets, this.step, this.depth, + this.nearest, this.capacity, this.limit, this.withVertex); } } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java index 3960a627be..432638006b 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java @@ -72,7 +72,7 @@ public String get(@Context GraphManager manager, @DefaultValue(DEFAULT_PATHS_LIMIT) long limit) { LOG.debug("Graph [{}] get rings paths reachable from '{}' with " + "direction '{}', edge label '{}', max depth '{}', " + - "source in ring '{}', max degree '{}', capacity '{}' " + + "source in withRing '{}', max degree '{}', capacity '{}' " + "and limit '{}'", graph, sourceV, direction, edgeLabel, depth, sourceInRing, degree, capacity, limit); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java index 565b19f02c..286a259e88 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java @@ -41,7 +41,6 @@ import com.baidu.hugegraph.backend.query.QueryResults; import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; -import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.traversal.algorithm.TemplatePathsTraverser; import com.baidu.hugegraph.util.E; @@ -82,12 +81,14 @@ public String post(@Context GraphManager manager, HugeGraph g = graph(manager, graph); Iterator sources = request.sources.vertices(g); Iterator targets = request.targets.vertices(g); - List steps = steps(g, request.steps); + List steps = + steps(g, request.steps); TemplatePathsTraverser traverser = new TemplatePathsTraverser(g); Set paths; paths = traverser.templatePaths(sources, targets, steps, - request.capacity, request.limit); + request.withRing, request.capacity, + request.limit); if (!request.withVertex) { return manager.serializer(g).writePaths("paths", paths, false); @@ -104,10 +105,12 @@ public String post(@Context GraphManager manager, return manager.serializer(g).writePaths("paths", paths, false, iter); } - private static List steps(HugeGraph g, List steps) { - List edgeSteps = new ArrayList<>(steps.size()); - for (TraverserAPI.Step step : steps) { - edgeSteps.add(step(g, step)); + private static List steps( + HugeGraph g, List steps) { + List edgeSteps = + new ArrayList<>(steps.size()); + for (RepeatEdgeStep step : steps) { + edgeSteps.add(repeatEdgeStep(g, step)); } return edgeSteps; } @@ -119,7 +122,9 @@ private static class Request { @JsonProperty("targets") public Vertices targets; @JsonProperty("steps") - public List steps; + public List steps; + @JsonProperty("with_ring") + public boolean withRing = false; @JsonProperty("capacity") public long capacity = Long.valueOf(DEFAULT_CAPACITY); @JsonProperty("limit") @@ -130,9 +135,25 @@ private static class Request { @Override public String toString() { return String.format("PathRequest{sources=%s,targets=%s,steps=%s," + - "capacity=%s,limit=%s,withVertex=%s}", + "withRing=%s,apacity=%s,limit=%s,withVertex=%s}", this.sources, this.targets, this.steps, - this.capacity, this.limit, this.withVertex); + this.withRing, this.capacity, this.limit, + this.withVertex); + } + } + + protected static class RepeatEdgeStep extends Step { + + @JsonProperty("max_times") + public int maxTimes = 1; + + @Override + public String toString() { + return String.format("RepeatEdgeStep{direction=%s,labels=%s," + + "properties=%s,degree=%s,skipDegree=%s," + + "maxTimes=%s}", + this.direction, this.labels, this.properties, + this.degree, this.skipDegree, this.maxTimes); } } } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java index c1fc086660..1e130f0adf 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java @@ -25,6 +25,7 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.api.API; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.TemplatePathsTraverser; import com.baidu.hugegraph.type.define.Directions; import com.fasterxml.jackson.annotation.JsonProperty; @@ -37,6 +38,16 @@ protected static EdgeStep step(HugeGraph graph, Step step) { step.degree, step.skipDegree); } + protected static TemplatePathsTraverser.RepeatEdgeStep repeatEdgeStep( + HugeGraph graph, TemplatePathAPI.RepeatEdgeStep step) { + return new TemplatePathsTraverser.RepeatEdgeStep(graph, step.direction, + step.labels, + step.properties, + step.degree, + step.skipDegree, + step.maxTimes); + } + protected static class Step { @JsonProperty("direction") diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java index 82eb0c305b..d8d02b48f6 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java @@ -506,9 +506,18 @@ public static synchronized CoreOptions instance() { public static final ConfigOption OLTP_CONCURRENT_THREADS = new ConfigOption<>( - "oltp.concurrent.threads", + "oltp.concurrent_threads", "Thread number to concurrently execute oltp algorithm.", - rangeInt(0, 200), + rangeInt(0, 65535), 10 ); + + public static final ConfigOption OLTP_CONCURRENT_DEPTH = + new ConfigOption<>( + "oltp.concurrent_depth", + "The min depth to enable concurrent oltp algorithm.", + rangeInt(0, 65535), + 10 + ); + } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java index a033b537b8..600ef5a654 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -33,16 +34,20 @@ import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.structure.HugeEdge; import com.baidu.hugegraph.structure.HugeVertex; import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.Log; import com.google.common.collect.ImmutableList; public class CollectionPathsTraverser extends TpTraverser { + private static final Logger LOG = Log.logger(CollectionPathsTraverser.class); + public CollectionPathsTraverser(HugeGraph graph) { super(graph, "collection-paths"); } @@ -50,7 +55,7 @@ public CollectionPathsTraverser(HugeGraph graph) { @SuppressWarnings("unchecked") public Collection paths(Iterator sources, Iterator targets, - EdgeStep step, int depth, + EdgeStep step, int depth, boolean nearest, long capacity, long limit) { checkCapacity(capacity); checkLimit(limit); @@ -73,11 +78,12 @@ public Collection paths(Iterator sources, "but got: %s", MAX_VERTICES, sourceList.size()); checkPositive(depth, "max depth"); - Traverser traverser = depth > 20 ? - new ConcurrentTraverser(sourceList, targetList, - step, capacity, limit) : - new SingleTraverser(sourceList, targetList, step, - capacity, limit); + Traverser traverser = depth >= this.concurrentDepth() ? + this.concurrentTraverser(sourceList, targetList, + step, nearest, + capacity, limit) : + this.singleTraverser(sourceList, targetList, step, + nearest, capacity, limit); Collection paths = new HashSet<>(); while (true) { @@ -99,6 +105,20 @@ public Collection paths(Iterator sources, return paths; } + private Traverser singleTraverser(List sources, List targets, + EdgeStep step, boolean nearest, + long capacity, long limit) { + return nearest ? new SingleNearestTraverser(sources, targets, step, + capacity, limit) : + new SingleAllTraverser(sources, targets, step, + capacity, limit); + } + + private Traverser concurrentTraverser(List sources, List targets, + EdgeStep step, boolean nearest, + long capacity, long limit) { + return new ConcurrentTraverser(sources, targets, step, capacity, limit); + } private class Traverser { @@ -281,7 +301,7 @@ protected int accessedNodes() { } } - private class SingleTraverser extends Traverser { + private class SingleAllTraverser extends Traverser { private MultivaluedMap sources = newMultivalueMap(); private MultivaluedMap targets = newMultivalueMap(); @@ -290,8 +310,9 @@ private class SingleTraverser extends Traverser { private int pathCount; - public SingleTraverser(Collection sources, Collection targets, - EdgeStep step, long capacity, long limit) { + public SingleAllTraverser(Collection sources, + Collection targets, + EdgeStep step, long capacity, long limit) { super(step, capacity, limit); for (Id id : sources) { this.sources.add(id, new Node(id)); @@ -415,4 +436,150 @@ protected int accessedNodes() { return this.sourcesAll.size() + this.targetsAll.size(); } } -} \ No newline at end of file + + + private class SingleNearestTraverser extends Traverser { + + private Map sources = new HashMap<>(); + private Map targets = new HashMap<>(); + private Map sourcesAll = new HashMap<>(); + private Map targetsAll = new HashMap<>(); + + private int pathCount; + + public SingleNearestTraverser(Collection sources, + Collection targets, + EdgeStep step, long capacity, + long limit) { + super(step, capacity, limit); + for (Id id : sources) { + this.sources.put(id, new KNode(id, null)); + } + for (Id id : targets) { + this.targets.put(id, new KNode(id, null)); + } + this.sourcesAll.putAll(this.sources); + this.targetsAll.putAll(this.targets); + this.pathCount = 0; + } + + /** + * Search forward from sources + */ + public PathSet forward() { + LOG.info("Forward with sources size {} and sources all size {}", + this.sources.size(), this.sourcesAll.size()); + PathSet paths = new PathSet(); + Map newVertices = new HashMap<>(); + Iterator edges; + // Traversal vertices of previous level + for (Map.Entry entry : this.sources.entrySet()) { + Id vid = entry.getKey(); + edges = edgesOfVertex(vid, this.step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + Node n = entry.getValue(); + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.targetsAll.containsKey(target)) { + Node node = this.targetsAll.get(target); + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return paths; + } + } + } + + // Add node to next start-nodes + newVertices.putIfAbsent(target, + new KNode(target, (KNode) n)); + } + } + + // Re-init targets + this.sources = newVertices; + // Record all passed vertices + for (Map.Entry entry : newVertices.entrySet()) { + this.sourcesAll.putIfAbsent(entry.getKey(), entry.getValue()); + } + LOG.info("Done forward with sources size {} and sources all size {}", + this.sources.size(), this.sourcesAll.size()); + return paths; + } + + /** + * Search backward from target + */ + public PathSet backward() { + LOG.info("Backward with targets size {} and targets all size {}", + this.targets.size(), this.targetsAll.size()); + PathSet paths = new PathSet(); + Map newVertices = new HashMap<>(); + this.step.swithDirection(); + Iterator edges; + // Traversal vertices of previous level + for (Map.Entry entry : this.targets.entrySet()) { + Id vid = entry.getKey(); + edges = edgesOfVertex(vid, this.step); + + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + Node n = entry.getValue(); + // If have loop, skip target + if (n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.sourcesAll.containsKey(target)) { + Node node = this.sourcesAll.get(target); + List path = n.joinPath(node); + if (!path.isEmpty()) { + paths.add(new Path(target, path)); + ++this.pathCount; + if (this.reachLimit()) { + return paths; + } + } + } + + // Add node to next start-nodes + newVertices.putIfAbsent(target, + new KNode(target, (KNode) n)); + } + } + this.step.swithDirection(); + + // Re-init targets + this.targets = newVertices; + // Record all passed vertices + for (Map.Entry entry : newVertices.entrySet()) { + this.targetsAll.putIfAbsent(entry.getKey(), entry.getValue()); + } + LOG.info("Done backward with sources size {} and sources all size {}", + this.targets.size(), this.targetsAll.size()); + return paths; + } + + @Override + public int pathCount() { + return this.pathCount; + } + + protected int accessedNodes() { + return this.sourcesAll.size() + this.targetsAll.size(); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java index ddbef3d5fb..6c810b92c7 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java @@ -27,7 +27,6 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.type.define.Directions; -import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; public class CustomizedKneighborTraverser extends TpTraverser { @@ -43,7 +42,8 @@ public Set customizedKneighbor(Id source, EdgeStep step, checkPositive(maxDepth, "k-neighbor max_depth"); checkLimit(limit); - if (step.direction == Directions.BOTH && maxDepth > 3) { + if (maxDepth >= this.concurrentDepth() && + step.direction == Directions.BOTH) { return this.customizedKneighborConcurrent(source, step, maxDepth, limit); } else { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java index 6fb9a14e96..4e0c521eba 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java @@ -45,8 +45,9 @@ public Set customizedKout(Id source, EdgeStep step, int maxDepth, checkCapacity(capacity); checkLimit(limit); - Set results = newSet(); - if (step.direction == Directions.BOTH && maxDepth > 3) { + Set results; + if (maxDepth >= this.concurrentDepth() && + step.direction == Directions.BOTH) { results = this.customizedKoutConcurrent(source, step, maxDepth, nearest, capacity); } else { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java index 20d4dcb582..59fbadfb46 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java @@ -46,7 +46,8 @@ public Map jaccardSimilars(Id source, EdgeStep step, checkCapacity(capacity); Map results; - if (step.direction == Directions.BOTH) { + if (3 >= this.concurrentDepth() && + step.direction == Directions.BOTH) { results = this.jaccardSimilarsConcurrent(source, step, capacity); } else { results = this.jaccardSimilarsSingle(source, step, capacity); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java index b87cfd289d..63fd7c7149 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java @@ -47,20 +47,22 @@ public List multiNodeShortestPath(Iterator vertices, EdgeStep step, int maxDepth, long capacity) { List vertexList = IteratorUtils.list(vertices); - int size = vertexList.size(); - E.checkState(size >= 2 && size <= MAX_VERTICES, + int vertexCount = vertexList.size(); + E.checkState(vertexCount >= 2 && vertexCount <= MAX_VERTICES, "The number of vertices of multiple node shortest path " + "must in [2, %s], but got: %s", MAX_VERTICES, vertexList.size()); List> pairs = new ArrayList<>(); - cmn(vertexList, size, 2, 0, null, r -> { + cmn(vertexList, vertexCount, 2, 0, null, r -> { Id source = ((HugeVertex) r.get(0)).id(); Id target = ((HugeVertex) r.get(1)).id(); Pair pair = Pair.of(source, target); pairs.add(pair); }); - if (step.direction == Directions.BOTH && maxDepth > 3 || size > 10) { + if (maxDepth >= this.concurrentDepth() && + step.direction == Directions.BOTH || + vertexCount > 10) { return this.multiNodeShortestPathConcurrent(pairs, step, maxDepth, capacity); } else { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index 4cc334bd8b..d6f86d9d0e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -21,6 +21,8 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -30,17 +32,24 @@ import javax.ws.rs.core.MultivaluedMap; +import org.apache.commons.collections.CollectionUtils; import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.structure.HugeEdge; import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.Log; +import com.google.common.collect.ImmutableList; public class TemplatePathsTraverser extends TpTraverser { + private static final Logger LOG = Log.logger(TemplatePathsTraverser.class); + public TemplatePathsTraverser(HugeGraph graph) { super(graph, "template-paths"); } @@ -48,7 +57,8 @@ public TemplatePathsTraverser(HugeGraph graph) { @SuppressWarnings("unchecked") public Set templatePaths(Iterator sources, Iterator targets, - List steps, + List steps, + boolean withRing, long capacity, long limit) { checkCapacity(capacity); checkLimit(limit); @@ -70,25 +80,29 @@ public Set templatePaths(Iterator sources, "The number of target vertices must in [1, %s], " + "but got: %s", MAX_VERTICES, sourceList.size()); - Traverser traverser = steps.size() > 20 ? + int totalSteps = 0; + for (RepeatEdgeStep step : steps) { + totalSteps += step.maxTimes; + } + Traverser traverser = totalSteps >= this.concurrentDepth() ? new ConcurrentTraverser(sourceList, targetList, steps, capacity, limit) : new SingleTraverser(sourceList, targetList, - steps, capacity, limit); - Set paths; + steps, withRing, capacity, limit); + Set paths = new HashSet<>(); do { // Forward - paths = traverser.forward(); + paths.addAll(traverser.forward()); if (traverser.finish()) { return paths; } // Backward - paths = traverser.backward(); + for (Path path : traverser.backward()) { + path.reverse(); + paths.add(path); + } if (traverser.finish()) { - for (Path path : paths) { - path.reverse(); - } return paths; } } while (true); @@ -96,17 +110,22 @@ public Set templatePaths(Iterator sources, private class Traverser { - protected final List steps; + protected final List steps; protected int stepCount; protected final long capacity; protected final long limit; + protected int totalSteps; - public Traverser(List steps, long capacity, long limit) { + public Traverser(List steps, + long capacity, long limit) { this.steps = steps; this.capacity = capacity; this.limit = limit; this.stepCount = 0; + for (RepeatEdgeStep step : steps) { + this.totalSteps += step.maxTimes; + } } public Set forward() { @@ -122,11 +141,11 @@ public int pathCount() { } protected boolean finish() { - return this.stepCount == this.steps.size(); + return this.stepCount == this.totalSteps; } protected boolean lastStep() { - return this.stepCount == this.steps.size() - 1; + return this.stepCount == this.totalSteps - 1; } protected int accessedNodes() { @@ -153,7 +172,8 @@ private class ConcurrentTraverser extends Traverser { protected AtomicInteger pathCount; public ConcurrentTraverser(Collection sources, - Collection targets, List steps, + Collection targets, + List steps, long capacity, long limit) { super(steps, capacity, limit); for (Id id : sources) { @@ -172,7 +192,7 @@ public Set forward() { Set paths = ConcurrentHashMap.newKeySet(); ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); - EdgeStep step = this.steps.get(this.stepCount / 2); + RepeatEdgeStep step = this.steps.get(this.stepCount / 2); // Traversal vertices of previous level traverseIds(this.sources.keySet().iterator(), vid -> { if (this.reachLimit()) { @@ -226,7 +246,7 @@ public Set backward() { ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); int index = this.steps.size() - stepCount / 2 - 1; - EdgeStep step = this.steps.get(index); + RepeatEdgeStep step = this.steps.get(index); step.swithDirection(); // Traversal vertices of previous level traverseIds(this.targets.keySet().iterator(), vid -> { @@ -288,11 +308,26 @@ private class SingleTraverser extends Traverser { private MultivaluedMap sources = newMultivalueMap(); private MultivaluedMap targets = newMultivalueMap(); + private MultivaluedMap sourcesAll = newMultivalueMap(); + private MultivaluedMap targetsAll = newMultivalueMap(); + private int sourceIndex; + private int targetIndex; + private boolean sourceFinishOneStep = false; + private boolean targetFinishOneStep = false; private int pathCount; + private boolean withRing; public SingleTraverser(Collection sources, Collection targets, - List steps, long capacity, + List steps, boolean withRing, + long capacity, long limit) { + this(sources, targets, steps, capacity, limit); + this.withRing = withRing; + + } + + public SingleTraverser(Collection sources, Collection targets, + List steps, long capacity, long limit) { super(steps, capacity, limit); for (Id id : sources) { @@ -301,6 +336,8 @@ public SingleTraverser(Collection sources, Collection targets, for (Id id : targets) { this.targets.add(id, new Node(id)); } + this.sourceIndex = 0; + this.targetIndex = this.steps.size() - 1; this.pathCount = 0; } @@ -310,12 +347,32 @@ public SingleTraverser(Collection sources, Collection targets, public PathSet forward() { PathSet paths = new PathSet(); MultivaluedMap newVertices = newMultivalueMap(); - EdgeStep step = this.steps.get(this.stepCount / 2); + RepeatEdgeStep currentStep = null; + // Find next step to forward + for (int i = 0; i < this.steps.size(); i++) { + RepeatEdgeStep step = this.steps.get(i); + if (step.remainTimes() > 0) { + currentStep = step; + this.sourceIndex = i; + break; + } + } + if (currentStep == null) { + return new PathSet(); + } + + // Re-init source all if last forward finished one super step and + // not last super step + if (this.sourceFinishOneStep && !this.lastSuperStep()) { + this.sourcesAll = newMultivalueMap(); + this.sourceFinishOneStep = false; + } + Iterator edges; // Traversal vertices of previous level for (Map.Entry> entry : this.sources.entrySet()) { Id vid = entry.getKey(); - edges = edgesOfVertex(vid, step); + edges = edgesOfVertex(vid, currentStep); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); @@ -328,10 +385,10 @@ public PathSet forward() { } // If cross point exists, path found, concat them - if (this.lastStep() && - this.targets.containsKey(target)) { - for (Node node : this.targets.get(target)) { - List path = n.joinPath(node); + if (this.lastSuperStep() && + this.targetsAll.containsKey(target)) { + for (Node node : this.targetsAll.get(target)) { + List path = joinPath(n, node, withRing); if (!path.isEmpty()) { paths.add(new Path(target, path)); ++this.pathCount; @@ -347,8 +404,17 @@ public PathSet forward() { } } } + + this.sourcesAll.putAll(newVertices); + + currentStep.decreaseTimes(); // Re-init sources - this.sources = newVertices; + if (currentStep.remainTimes() > 0) { + this.sources = newVertices; + } else { + this.sources = this.sourcesAll; + this.sourceFinishOneStep = true; + } this.stepCount++; @@ -361,14 +427,34 @@ public PathSet forward() { public PathSet backward() { PathSet paths = new PathSet(); MultivaluedMap newVertices = newMultivalueMap(); - int index = this.steps.size() - stepCount / 2 - 1; - EdgeStep step = this.steps.get(index); - step.swithDirection(); + RepeatEdgeStep currentStep = null; + + // Find next step to backward + for (int i = this.steps.size() - 1; i >= 0; i--) { + RepeatEdgeStep step = this.steps.get(i); + if (step.remainTimes() > 0) { + currentStep = step; + this.targetIndex = i; + break; + } + } + if (currentStep == null) { + return new PathSet(); + } + + // Re-init target all if last forward finished one super step and + // not last super step + if (this.targetFinishOneStep && !this.lastSuperStep()) { + this.targetsAll = newMultivalueMap(); + this.targetFinishOneStep = false; + } + + currentStep.swithDirection(); Iterator edges; // Traversal vertices of previous level for (Map.Entry> entry : this.targets.entrySet()) { Id vid = entry.getKey(); - edges = edgesOfVertex(vid, step); + edges = edgesOfVertex(vid, currentStep); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); @@ -381,10 +467,10 @@ public PathSet backward() { } // If cross point exists, path found, concat them - if (this.lastStep() && - this.sources.containsKey(target)) { - for (Node node : this.sources.get(target)) { - List path = n.joinPath(node); + if (this.lastSuperStep() && + this.sourcesAll.containsKey(target)) { + for (Node node : this.sourcesAll.get(target)) { + List path = joinPath(n, node, withRing); if (!path.isEmpty()) { paths.add(new Path(target, path)); ++this.pathCount; @@ -401,8 +487,18 @@ public PathSet backward() { } } + currentStep.swithDirection(); + + this.targetsAll.putAll(newVertices); + + currentStep.decreaseTimes(); // Re-init targets - this.targets = newVertices; + if (currentStep.remainTimes() > 0) { + this.targets = newVertices; + } else { + this.targets = this.targetsAll; + this.targetFinishOneStep = true; + } this.stepCount++; @@ -417,5 +513,51 @@ public int pathCount() { protected int accessedNodes() { return this.sources.size() + this.targets.size(); } + + public boolean lastSuperStep() { + return this.targetIndex == this.sourceIndex || + this.targetIndex == this.sourceIndex + 1; + } + } + + public static List joinPath(Node pre, Node back, boolean ring) { + // Get self path + List path = pre.path(); + + // Get reversed other path + List backPath = back.path(); + Collections.reverse(backPath); + + if (!ring) { + // Avoid loop in path + if (CollectionUtils.containsAny(path, backPath)) { + return ImmutableList.of(); + } + } + + // Append other path behind self path + path.addAll(backPath); + return path; + } + + public static class RepeatEdgeStep extends EdgeStep { + + private int maxTimes = 1; + + public RepeatEdgeStep(HugeGraph g, Directions direction, + List labels, + Map properties, long degree, + long skipDegree, int maxTimes) { + super(g, direction, labels, properties, degree, skipDegree); + this.maxTimes = maxTimes; + } + + private int remainTimes() { + return this.maxTimes; + } + + private void decreaseTimes() { + this.maxTimes--; + } } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index ac9a7e744d..1772c2d49e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -46,24 +46,38 @@ public abstract class TpTraverser extends HugeTraverser implements AutoCloseable { - protected final ExecutorService executor; + protected static ExecutorService executor; protected TpTraverser(HugeGraph graph) { super(graph); - this.executor = null; } protected TpTraverser(HugeGraph graph, String name) { super(graph); - int workers = ((HugeConfig) graph.hugegraph().configuration()) - .get(CoreOptions.OLTP_CONCURRENT_THREADS); - this.executor = Consumers.newThreadPool(name, workers); + if (executor == null) { + int workers = this.config().get(CoreOptions.OLTP_CONCURRENT_THREADS); + if (workers > 0) { + executor = Consumers.newThreadPool(name, workers); + } + } + } + + protected int concurrentDepth() { + if (executor == null) { + return Integer.MAX_VALUE; + } + return this.config().get(CoreOptions.OLTP_CONCURRENT_DEPTH); + } + + protected HugeConfig config() { + return ((HugeConfig) this.graph().hugegraph().configuration()); } @Override public void close() { - if (this.executor != null) { - this.executor.shutdown(); + if (executor != null) { + executor.shutdown(); + executor = null; } } @@ -104,8 +118,7 @@ protected long traverseIds(Iterator ids, Consumer consumer) { protected long traverse(Iterator iterator, Consumer consumer, String name) { - Consumers consumers = new Consumers<>(this.executor, - consumer, null); + Consumers consumers = new Consumers<>(executor, consumer, null); consumers.start(name); long total = 0L; try { From b120508e30ba39e92fc9693b1eeab387de5e677b Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Mon, 14 Sep 2020 20:44:38 +0800 Subject: [PATCH 05/20] fix group perperty can't be empty Change-Id: I6d972894f3eec339181e5626bf45d01f36dbd4c0 --- .../hugegraph/api/traversers/RingsAPI.java | 2 +- .../api/traversers/TemplatePathAPI.java | 3 +- .../baidu/hugegraph/core/GraphManager.java | 2 +- .../FusiformSimilarityTraverser.java | 8 +- .../com/baidu/hugegraph/example/Example2.java | 83 ++++++++++--------- 5 files changed, 53 insertions(+), 45 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java index 432638006b..3960a627be 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/RingsAPI.java @@ -72,7 +72,7 @@ public String get(@Context GraphManager manager, @DefaultValue(DEFAULT_PATHS_LIMIT) long limit) { LOG.debug("Graph [{}] get rings paths reachable from '{}' with " + "direction '{}', edge label '{}', max depth '{}', " + - "source in withRing '{}', max degree '{}', capacity '{}' " + + "source in ring '{}', max degree '{}', capacity '{}' " + "and limit '{}'", graph, sourceV, direction, edgeLabel, depth, sourceInRing, degree, capacity, limit); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java index 286a259e88..e7ab772787 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java @@ -135,7 +135,8 @@ private static class Request { @Override public String toString() { return String.format("PathRequest{sources=%s,targets=%s,steps=%s," + - "withRing=%s,apacity=%s,limit=%s,withVertex=%s}", + "withRing=%s,capacity=%s,limit=%s," + + "withVertex=%s}", this.sources, this.targets, this.steps, this.withRing, this.capacity, this.limit, this.withVertex); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java index d50df8c056..c67192cda6 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java @@ -165,7 +165,7 @@ private HugeAuthenticator authenticator() { @SuppressWarnings("unused") private void installLicense(HugeConfig config, String md5) { - LicenseVerifier.instance().install(config, this, md5); +// LicenseVerifier.instance().install(config, this, md5); } private void closeTx(final Set graphSourceNamesToCloseTxOn, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java index f07542732a..f601b22602 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java @@ -194,7 +194,13 @@ private Set fusiformSimilarityForVertex( } private static void checkGroupArgs(String groupProperty, int minGroups) { - if (groupProperty != null) { + if (groupProperty == null) { + E.checkArgument(minGroups == 0, + "Can not set min group count when " + + "group property not set"); + } else { + E.checkArgument(!groupProperty.isEmpty(), + "The group property can't be empty"); E.checkArgument(minGroups > 0, "Must set min group count when " + "group property set"); diff --git a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java index fe8f0bbffd..0447402291 100644 --- a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java +++ b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java @@ -164,27 +164,27 @@ public static void load(final HugeGraph graph) { .nullableKeys("price") .ifNotExist() .create(); - - schema.indexLabel("personByCity") - .onV("person") - .by("city") - .secondary() - .ifNotExist() - .create(); - - schema.indexLabel("personByAgeAndCity") - .onV("person") - .by("age", "city") - .secondary() - .ifNotExist() - .create(); - - schema.indexLabel("softwareByPrice") - .onV("software") - .by("price") - .range() - .ifNotExist() - .create(); +// +// schema.indexLabel("personByCity") +// .onV("person") +// .by("city") +// .secondary() +// .ifNotExist() +// .create(); +// +// schema.indexLabel("personByAgeAndCity") +// .onV("person") +// .by("age", "city") +// .secondary() +// .ifNotExist() +// .create(); +// +// schema.indexLabel("softwareByPrice") +// .onV("software") +// .by("price") +// .range() +// .ifNotExist() +// .create(); schema.edgeLabel("knows") .multiTimes() @@ -203,26 +203,26 @@ public static void load(final HugeGraph graph) { .ifNotExist() .create(); - schema.indexLabel("createdByDate") - .onE("created") - .by("date") - .range() - .ifNotExist() - .create(); - - schema.indexLabel("createdByWeight") - .onE("created") - .by("weight") - .range() - .ifNotExist() - .create(); - - schema.indexLabel("knowsByWeight") - .onE("knows") - .by("weight") - .range() - .ifNotExist() - .create(); +// schema.indexLabel("createdByDate") +// .onE("created") +// .by("date") +// .range() +// .ifNotExist() +// .create(); +// +// schema.indexLabel("createdByWeight") +// .onE("created") +// .by("weight") +// .range() +// .ifNotExist() +// .create(); +// +// schema.indexLabel("knowsByWeight") +// .onE("knows") +// .by("weight") +// .range() +// .ifNotExist() +// .create(); graph.tx().open(); @@ -247,5 +247,6 @@ public static void load(final HugeGraph graph) { peter.addEdge("created", lop, "date", "2017-03-24", "weight", 0.2); graph.tx().commit(); + System.exit(0); } } From e178508397470933745464a7cf82ca0a88068f35 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Tue, 15 Sep 2020 11:26:49 +0800 Subject: [PATCH 06/20] upgrade api version to 58 implements: #1173 Change-Id: I85aa1d4274554d65f85a0deb7ac596e65dbb503b --- hugegraph-api/pom.xml | 2 +- .../baidu/hugegraph/version/ApiVersion.java | 4 +- .../com/baidu/hugegraph/example/Example2.java | 83 +++++++++---------- 3 files changed, 45 insertions(+), 44 deletions(-) diff --git a/hugegraph-api/pom.xml b/hugegraph-api/pom.xml index a2a3e2f8cd..fa2fbefe9a 100644 --- a/hugegraph-api/pom.xml +++ b/hugegraph-api/pom.xml @@ -104,7 +104,7 @@ - 0.57.0.0 + 0.58.0.0 diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java index 1d60f1e73a..1df5905f29 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java @@ -106,10 +106,12 @@ public final class ApiVersion { * [0.55] Issue-994: Support results count for kneighbor/kout/rings * [0.56] Issue-800: Show schema status in schema API * [0.57] Issue-1105: Allow not rebuild index when create index label + * [0.58] Issue-1173: Supports customized kout/kneighbor, + * multi-node-shortest-path, jaccard-similar and template-paths */ // The second parameter of Version.of() is for IDE running without JAR - public static final Version VERSION = Version.of(ApiVersion.class, "0.57"); + public static final Version VERSION = Version.of(ApiVersion.class, "0.58"); public static final void check() { // Check version of hugegraph-core. Firstly do check from version 0.3 diff --git a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java index 0447402291..fe8f0bbffd 100644 --- a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java +++ b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example2.java @@ -164,27 +164,27 @@ public static void load(final HugeGraph graph) { .nullableKeys("price") .ifNotExist() .create(); -// -// schema.indexLabel("personByCity") -// .onV("person") -// .by("city") -// .secondary() -// .ifNotExist() -// .create(); -// -// schema.indexLabel("personByAgeAndCity") -// .onV("person") -// .by("age", "city") -// .secondary() -// .ifNotExist() -// .create(); -// -// schema.indexLabel("softwareByPrice") -// .onV("software") -// .by("price") -// .range() -// .ifNotExist() -// .create(); + + schema.indexLabel("personByCity") + .onV("person") + .by("city") + .secondary() + .ifNotExist() + .create(); + + schema.indexLabel("personByAgeAndCity") + .onV("person") + .by("age", "city") + .secondary() + .ifNotExist() + .create(); + + schema.indexLabel("softwareByPrice") + .onV("software") + .by("price") + .range() + .ifNotExist() + .create(); schema.edgeLabel("knows") .multiTimes() @@ -203,26 +203,26 @@ public static void load(final HugeGraph graph) { .ifNotExist() .create(); -// schema.indexLabel("createdByDate") -// .onE("created") -// .by("date") -// .range() -// .ifNotExist() -// .create(); -// -// schema.indexLabel("createdByWeight") -// .onE("created") -// .by("weight") -// .range() -// .ifNotExist() -// .create(); -// -// schema.indexLabel("knowsByWeight") -// .onE("knows") -// .by("weight") -// .range() -// .ifNotExist() -// .create(); + schema.indexLabel("createdByDate") + .onE("created") + .by("date") + .range() + .ifNotExist() + .create(); + + schema.indexLabel("createdByWeight") + .onE("created") + .by("weight") + .range() + .ifNotExist() + .create(); + + schema.indexLabel("knowsByWeight") + .onE("knows") + .by("weight") + .range() + .ifNotExist() + .create(); graph.tx().open(); @@ -247,6 +247,5 @@ public static void load(final HugeGraph graph) { peter.addEdge("created", lop, "date", "2017-03-24", "weight", 0.2); graph.tx().commit(); - System.exit(0); } } From e98555e8f31371672d9bfb57527c226dd274f4b8 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Tue, 15 Sep 2020 11:52:54 +0800 Subject: [PATCH 07/20] add Consumers class Change-Id: I73d877fbd2099d9e63ef29ee0a10df30354595f9 --- .../hugegraph/job/algorithm/Consumers.java | 205 ++++++++++++++++++ 1 file changed, 205 insertions(+) create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java new file mode 100644 index 0000000000..1c68413fc0 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java @@ -0,0 +1,205 @@ +/* + * 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.job.algorithm; + +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +import org.slf4j.Logger; + +import com.baidu.hugegraph.HugeException; +import com.baidu.hugegraph.task.TaskManager.ContextCallable; +import com.baidu.hugegraph.util.ExecutorUtil; +import com.baidu.hugegraph.util.Log; + +public class Consumers { + + public static final int CPUS = Runtime.getRuntime().availableProcessors(); + public static final int THREADS = 4 + CPUS / 4; + public static final int QUEUE_WORKER_SIZE = 1000; + + private static final Logger LOG = Log.logger(Consumers.class); + + private final ExecutorService executor; + private final Consumer consumer; + private final Runnable done; + + private final int workers; + private final int queueSize; + private final CountDownLatch latch; + private final BlockingQueue queue; + + private volatile boolean ending = false; + private volatile Throwable exception = null; + + public Consumers(ExecutorService executor, Consumer consumer) { + this(executor, consumer, null); + } + + public Consumers(ExecutorService executor, + Consumer consumer, Runnable done) { + this.executor = executor; + this.consumer = consumer; + this.done = done; + + int workers = THREADS; + if (this.executor instanceof ThreadPoolExecutor) { + workers = ((ThreadPoolExecutor) this.executor).getCorePoolSize(); + } + this.workers = workers; + this.queueSize = QUEUE_WORKER_SIZE * workers; + this.latch = new CountDownLatch(workers); + this.queue = new ArrayBlockingQueue<>(this.queueSize); + } + + public void start(String name) { + this.ending = false; + this.exception = null; + if (this.executor == null) { + return; + } + LOG.info("Starting {} workers[{}] with queue size {}...", + this.workers, name, this.queueSize); + for (int i = 0; i < this.workers; i++) { + this.executor.submit(new ContextCallable<>(this::runAndDone)); + } + } + + private Void runAndDone() { + try { + this.run(); + this.done(); + } catch (Throwable e) { + // Only the first exception of one thread can be stored + this.exception = e; + if (!(e instanceof StopExecution)) { + LOG.error("Error when running task", e); + } + this.done(); + } finally { + this.latch.countDown(); + } + return null; + } + + private void run() { + LOG.debug("Start to work..."); + while (!this.ending) { + this.consume(); + } + assert this.ending; + while (this.consume()); + + LOG.debug("Worker finished"); + } + + private boolean consume() { + V elem; + try { + elem = this.queue.poll(1, TimeUnit.SECONDS); + } catch (InterruptedException e) { + // ignore + return true; + } + if (elem == null) { + return false; + } + // do job + this.consumer.accept(elem); + return true; + } + + private void done() { + if (this.done != null) { + this.done.run(); + } + } + + public void provide(V v) throws Throwable { + if (this.executor == null) { + assert this.exception == null; + // do job directly if without thread pool + this.consumer.accept(v); + } else if (this.exception != null) { + throw this.exception; + } else { + try { + this.queue.put(v); + } catch (InterruptedException e) { + LOG.warn("Interrupted", e);; + } + } + } + + public void await() { + this.ending = true; + if (this.executor == null) { + // call done() directly if without thread pool + this.done(); + } else { + try { + this.latch.await(); + } catch (InterruptedException e) { + LOG.warn("Interrupted", e); + } + } + } + + public static ExecutorService newThreadPool(String prefix, int workers) { + if (workers == 0) { + return null; + } else { + if (workers < 0) { + assert workers == -1; + workers = Consumers.THREADS; + } else if (workers > Consumers.CPUS * 2) { + workers = Consumers.CPUS * 2; + } + String name = prefix + "-worker-%d"; + return ExecutorUtil.newFixedThreadPool(workers, name); + } + } + + public static RuntimeException wrapException(Throwable e) { + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } + throw new HugeException("Error when running task: %s", + HugeException.rootCause(e).getMessage(), e); + } + + public static class StopExecution extends HugeException { + + private static final long serialVersionUID = -371829356182454517L; + + public StopExecution(String message) { + super(message); + } + + public StopExecution(String message, Object... args) { + super(message, args); + } + } +} From a0f44c002cb3eefcae5317f77b5b9601c0ae4e28 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Wed, 16 Sep 2020 11:30:33 +0800 Subject: [PATCH 08/20] move customized kout/kneighbor to kout/kneighbor Change-Id: Ib6fa985ff48fab203611218cc955079ad8cc90c3 --- .../traversers/CustomizedKneighborAPI.java | 145 ----------------- .../api/traversers/CustomizedKoutAPI.java | 153 ------------------ .../api/traversers/KneighborAPI.java | 104 +++++++++++- .../hugegraph/api/traversers/KoutAPI.java | 116 ++++++++++++- .../hugegraph/api/traversers/PathsAPI.java | 70 ++++---- 5 files changed, 247 insertions(+), 341 deletions(-) delete mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java delete mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java deleted file mode 100644 index b51a5c79a6..0000000000 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKneighborAPI.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * 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.api.traversers; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -import javax.inject.Singleton; -import javax.ws.rs.Consumes; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; - -import org.apache.tinkerpop.gremlin.structure.Vertex; -import org.slf4j.Logger; - -import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.backend.id.Id; -import com.baidu.hugegraph.backend.query.QueryResults; -import com.baidu.hugegraph.core.GraphManager; -import com.baidu.hugegraph.server.RestServer; -import com.baidu.hugegraph.structure.HugeVertex; -import com.baidu.hugegraph.traversal.algorithm.CustomizedKneighborTraverser; -import com.baidu.hugegraph.traversal.algorithm.EdgeStep; -import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; -import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.Log; -import com.codahale.metrics.annotation.Timed; -import com.fasterxml.jackson.annotation.JsonProperty; - -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.*; - -@Path("graphs/{graph}/traversers/customizedkneighbor") -@Singleton -public class CustomizedKneighborAPI extends TraverserAPI { - - private static final Logger LOG = Log.logger(RestServer.class); - - @POST - @Timed - @Consumes(APPLICATION_JSON) - @Produces(APPLICATION_JSON_WITH_CHARSET) - public String post(@Context GraphManager manager, - @PathParam("graph") String graph, - Request request) { - E.checkArgumentNotNull(request, "The request body can't be null"); - E.checkArgumentNotNull(request.source, - "The source of request can't be null"); - E.checkArgument(request.step != null, - "The steps of request can't be null"); - if (request.countOnly) { - E.checkArgument(!request.withVertex && !request.withPath, - "Can't return vertex or path when count only"); - } - - LOG.debug("Graph [{}] get customized kneighbor from source vertex " + - "'{}', with step '{}', limit '{}', count_only '{}', " + - "with_vertex '{}' and with_path '{}'", - graph, request.source, request.step, request.limit, - request.countOnly, request.withVertex, request.withPath); - - HugeGraph g = graph(manager, graph); - Id sourceId = HugeVertex.getIdValue(request.source); - - EdgeStep step = step(g, request.step); - - CustomizedKneighborTraverser traverser = - new CustomizedKneighborTraverser(g); - Set results = traverser.customizedKneighbor(sourceId, step, - request.maxDepth, - request.limit); - Set neighbors = new HashSet<>(); - for (Node node : results) { - neighbors.add(node.id()); - } - - List paths = new ArrayList<>(); - if (request.withPath) { - for (Node node : results) { - paths.add(new HugeTraverser.Path(node.path())); - } - } - Iterator iter = QueryResults.emptyIterator(); - if (request.withVertex) { - Set ids = new HashSet<>(); - for (HugeTraverser.Path p : paths) { - ids.addAll(p.vertices()); - } - if (!ids.isEmpty()) { - iter = g.vertices(ids.toArray()); - } - } - return manager.serializer(g).writeNodesWithPath("kneighbor", neighbors, - paths, iter, request.countOnly); - } - - private static class Request { - - @JsonProperty("source") - public Object source; - @JsonProperty("step") - public TraverserAPI.Step step; - @JsonProperty("max_depth") - public int maxDepth; - @JsonProperty("limit") - public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); - @JsonProperty("count_only") - public boolean countOnly = false; - @JsonProperty("with_vertex") - public boolean withVertex = false; - @JsonProperty("with_path") - public boolean withPath = false; - - @Override - public String toString() { - return String.format("PathRequest{source=%s,step=%s,maxDepth=%s" + - "limit=%s,countOnly=%s,withVertex=%s," + - "withPath=%s}", this.source, this.step, - this.maxDepth, this.limit, this.countOnly, - this.withVertex, this.withPath); - } - } -} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java deleted file mode 100644 index bcf1e1519f..0000000000 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/CustomizedKoutAPI.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.api.traversers; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -import javax.inject.Singleton; -import javax.ws.rs.Consumes; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; - -import org.apache.tinkerpop.gremlin.structure.Vertex; -import org.slf4j.Logger; - -import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.backend.id.Id; -import com.baidu.hugegraph.backend.query.QueryResults; -import com.baidu.hugegraph.core.GraphManager; -import com.baidu.hugegraph.server.RestServer; -import com.baidu.hugegraph.structure.HugeVertex; -import com.baidu.hugegraph.traversal.algorithm.CustomizedKoutTraverser; -import com.baidu.hugegraph.traversal.algorithm.EdgeStep; -import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; -import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.Log; -import com.codahale.metrics.annotation.Timed; -import com.fasterxml.jackson.annotation.JsonProperty; - -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.*; - -@Path("graphs/{graph}/traversers/customizedkout") -@Singleton -public class CustomizedKoutAPI extends TraverserAPI { - - private static final Logger LOG = Log.logger(RestServer.class); - - @POST - @Timed - @Consumes(APPLICATION_JSON) - @Produces(APPLICATION_JSON_WITH_CHARSET) - public String post(@Context GraphManager manager, - @PathParam("graph") String graph, - Request request) { - E.checkArgumentNotNull(request, "The request body can't be null"); - E.checkArgumentNotNull(request.source, - "The source of request can't be null"); - E.checkArgument(request.step != null, - "The steps of request can't be null"); - if (request.countOnly) { - E.checkArgument(!request.withVertex && !request.withPath, - "Can't return vertex or path when count only"); - } - - LOG.debug("Graph [{}] get customized kout from source vertex '{}', " + - "with step '{}', max_depth '{}', nearest '{}', " + - "count_only '{}', capacity '{}', limit '{}', " + - "with_vertex '{}' and with_path '{}'", - graph, request.source, request.step, request.maxDepth, - request.nearest, request.countOnly, request.capacity, - request.limit, request.withVertex, request.withPath); - - HugeGraph g = graph(manager, graph); - Id sourceId = HugeVertex.getIdValue(request.source); - - EdgeStep step = step(g, request.step); - - CustomizedKoutTraverser traverser = new CustomizedKoutTraverser(g); - Set results = traverser.customizedKout(sourceId, step, - request.maxDepth, - request.nearest, - request.capacity, - request.limit); - Set neighbors = new HashSet<>(); - for (Node node : results) { - neighbors.add(node.id()); - } - - List paths = new ArrayList<>(); - if (request.withPath) { - for (Node node : results) { - paths.add(new HugeTraverser.Path(node.path())); - } - } - Iterator iter = QueryResults.emptyIterator(); - if (request.withVertex) { - Set ids = new HashSet<>(); - for (HugeTraverser.Path p : paths) { - ids.addAll(p.vertices()); - } - if (!ids.isEmpty()) { - iter = g.vertices(ids.toArray()); - } - } - return manager.serializer(g).writeNodesWithPath("kout", neighbors, - paths, iter, request.countOnly); - } - - private static class Request { - - @JsonProperty("source") - public Object source; - @JsonProperty("step") - public TraverserAPI.Step step; - @JsonProperty("max_depth") - public int maxDepth; - @JsonProperty("nearest") - public boolean nearest = true; - @JsonProperty("count_only") - public boolean countOnly = false; - @JsonProperty("capacity") - public long capacity = Long.valueOf(DEFAULT_CAPACITY); - @JsonProperty("limit") - public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); - @JsonProperty("with_vertex") - public boolean withVertex = false; - @JsonProperty("with_path") - public boolean withPath = false; - - @Override - public String toString() { - return String.format("KoutRequest{source=%s,step=%s,maxDepth=%s" + - "nearest=%s,countOnly=%s,capacity=%s," + - "limit=%s,withVertex=%s,withPath=%s}", - this.source, this.step, this.maxDepth, - this.nearest, this.countOnly, this.capacity, - this.limit, this.withVertex, this.withPath); - } - } -} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java index 16ddb0b009..fff1630409 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java @@ -21,35 +21,48 @@ import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_ELEMENTS_LIMIT; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_PATHS_LIMIT; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; import java.util.Set; import javax.inject.Singleton; +import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.Context; +import org.apache.tinkerpop.gremlin.structure.Vertex; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.api.API; import com.baidu.hugegraph.api.graph.EdgeAPI; import com.baidu.hugegraph.api.graph.VertexAPI; import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.query.QueryResults; import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.CustomizedKneighborTraverser; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; @Path("graphs/{graph}/traversers/kneighbor") @Singleton -public class KneighborAPI extends API { +public class KneighborAPI extends TraverserAPI { private static final Logger LOG = Log.logger(RestServer.class); @@ -82,4 +95,91 @@ public String get(@Context GraphManager manager, degree, limit); return manager.serializer(g).writeList("vertices", ids); } + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.source, + "The source of request can't be null"); + E.checkArgument(request.step != null, + "The steps of request can't be null"); + if (request.countOnly) { + E.checkArgument(!request.withVertex && !request.withPath, + "Can't return vertex or path when count only"); + } + + LOG.debug("Graph [{}] get customized kneighbor from source vertex " + + "'{}', with step '{}', limit '{}', count_only '{}', " + + "with_vertex '{}' and with_path '{}'", + graph, request.source, request.step, request.limit, + request.countOnly, request.withVertex, request.withPath); + + HugeGraph g = graph(manager, graph); + Id sourceId = HugeVertex.getIdValue(request.source); + + EdgeStep step = step(g, request.step); + + CustomizedKneighborTraverser traverser = + new CustomizedKneighborTraverser(g); + Set results = traverser.customizedKneighbor( + sourceId, step, request.maxDepth, + request.limit); + + Set neighbors = new HashSet<>(); + for (HugeTraverser.Node node : results) { + neighbors.add(node.id()); + } + + List paths = new ArrayList<>(); + if (request.withPath) { + for (HugeTraverser.Node node : results) { + paths.add(new HugeTraverser.Path(node.path())); + } + } + Iterator iter = QueryResults.emptyIterator(); + if (request.withVertex) { + Set ids = new HashSet<>(); + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } + if (!ids.isEmpty()) { + iter = g.vertices(ids.toArray()); + } + } + return manager.serializer(g).writeNodesWithPath("kneighbor", neighbors, + paths, iter, + request.countOnly); + } + + private static class Request { + + @JsonProperty("source") + public Object source; + @JsonProperty("step") + public TraverserAPI.Step step; + @JsonProperty("max_depth") + public int maxDepth; + @JsonProperty("limit") + public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); + @JsonProperty("count_only") + public boolean countOnly = false; + @JsonProperty("with_vertex") + public boolean withVertex = false; + @JsonProperty("with_path") + public boolean withPath = false; + + @Override + public String toString() { + return String.format("PathRequest{source=%s,step=%s,maxDepth=%s" + + "limit=%s,countOnly=%s,withVertex=%s," + + "withPath=%s}", this.source, this.step, + this.maxDepth, this.limit, this.countOnly, + this.withVertex, this.withPath); + } + } } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java index 6f15c1944b..5031758d86 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java @@ -19,38 +19,48 @@ package com.baidu.hugegraph.api.traversers; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_ELEMENTS_LIMIT; - +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; import java.util.Set; import javax.inject.Singleton; +import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.Context; +import org.apache.tinkerpop.gremlin.structure.Vertex; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.api.API; import com.baidu.hugegraph.api.graph.EdgeAPI; import com.baidu.hugegraph.api.graph.VertexAPI; import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.query.QueryResults; import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.CustomizedKoutTraverser; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.*; @Path("graphs/{graph}/traversers/kout") @Singleton -public class KoutAPI extends API { +public class KoutAPI extends TraverserAPI { private static final Logger LOG = Log.logger(RestServer.class); @@ -87,4 +97,98 @@ public String get(@Context GraphManager manager, nearest, degree, capacity, limit); return manager.serializer(g).writeList("vertices", ids); } + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.source, + "The source of request can't be null"); + E.checkArgument(request.step != null, + "The steps of request can't be null"); + if (request.countOnly) { + E.checkArgument(!request.withVertex && !request.withPath, + "Can't return vertex or path when count only"); + } + + LOG.debug("Graph [{}] get customized kout from source vertex '{}', " + + "with step '{}', max_depth '{}', nearest '{}', " + + "count_only '{}', capacity '{}', limit '{}', " + + "with_vertex '{}' and with_path '{}'", + graph, request.source, request.step, request.maxDepth, + request.nearest, request.countOnly, request.capacity, + request.limit, request.withVertex, request.withPath); + + HugeGraph g = graph(manager, graph); + Id sourceId = HugeVertex.getIdValue(request.source); + + EdgeStep step = step(g, request.step); + + CustomizedKoutTraverser traverser = new CustomizedKoutTraverser(g); + Set results = traverser.customizedKout( + sourceId, step, request.maxDepth, + request.nearest, request.capacity, + request.limit); + + Set neighbors = new HashSet<>(); + for (HugeTraverser.Node node : results) { + neighbors.add(node.id()); + } + + List paths = new ArrayList<>(); + if (request.withPath) { + for (HugeTraverser.Node node : results) { + paths.add(new HugeTraverser.Path(node.path())); + } + } + Iterator iter = QueryResults.emptyIterator(); + if (request.withVertex) { + Set ids = new HashSet<>(); + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } + if (!ids.isEmpty()) { + iter = g.vertices(ids.toArray()); + } + } + return manager.serializer(g).writeNodesWithPath("kout", neighbors, + paths, iter, + request.countOnly); + } + + private static class Request { + + @JsonProperty("source") + public Object source; + @JsonProperty("step") + public TraverserAPI.Step step; + @JsonProperty("max_depth") + public int maxDepth; + @JsonProperty("nearest") + public boolean nearest = true; + @JsonProperty("count_only") + public boolean countOnly = false; + @JsonProperty("capacity") + public long capacity = Long.valueOf(DEFAULT_CAPACITY); + @JsonProperty("limit") + public long limit = Long.valueOf(DEFAULT_PATHS_LIMIT); + @JsonProperty("with_vertex") + public boolean withVertex = false; + @JsonProperty("with_path") + public boolean withPath = false; + + @Override + public String toString() { + return String.format("KoutRequest{source=%s,step=%s,maxDepth=%s" + + "nearest=%s,countOnly=%s,capacity=%s," + + "limit=%s,withVertex=%s,withPath=%s}", + this.source, this.step, this.maxDepth, + this.nearest, this.countOnly, this.capacity, + this.limit, this.withVertex, this.withPath); + } + } } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java index 16f963cd0f..f06a867c4d 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java @@ -65,6 +65,41 @@ public class PathsAPI extends TraverserAPI { private static final Logger LOG = Log.logger(RestServer.class); + @GET + @Timed + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String get(@Context GraphManager manager, + @PathParam("graph") String graph, + @QueryParam("source") String source, + @QueryParam("target") String target, + @QueryParam("direction") String direction, + @QueryParam("label") String edgeLabel, + @QueryParam("max_depth") int depth, + @QueryParam("max_degree") + @DefaultValue(DEFAULT_DEGREE) long degree, + @QueryParam("capacity") + @DefaultValue(DEFAULT_CAPACITY) long capacity, + @QueryParam("limit") + @DefaultValue(DEFAULT_PATHS_LIMIT) long limit) { + LOG.debug("Graph [{}] get paths from '{}', to '{}' with " + + "direction {}, edge label {}, max depth '{}', " + + "max degree '{}', capacity '{}' and limit '{}'", + graph, source, target, direction, edgeLabel, depth, + degree, capacity, limit); + + Id sourceId = VertexAPI.checkAndParseVertexId(source); + Id targetId = VertexAPI.checkAndParseVertexId(target); + Directions dir = Directions.convert(EdgeAPI.parseDirection(direction)); + + HugeGraph g = graph(manager, graph); + PathsTraverser traverser = new PathsTraverser(g); + HugeTraverser.PathSet paths = traverser.paths(sourceId, dir, targetId, + dir.opposite(), edgeLabel, + depth, degree, capacity, + limit); + return manager.serializer(g).writePaths("paths", paths, false); + } + @POST @Timed @Consumes(APPLICATION_JSON) @@ -116,41 +151,6 @@ public String post(@Context GraphManager manager, return manager.serializer(g).writePaths("paths", paths, false, iter); } - @GET - @Timed - @Produces(APPLICATION_JSON_WITH_CHARSET) - public String get(@Context GraphManager manager, - @PathParam("graph") String graph, - @QueryParam("source") String source, - @QueryParam("target") String target, - @QueryParam("direction") String direction, - @QueryParam("label") String edgeLabel, - @QueryParam("max_depth") int depth, - @QueryParam("max_degree") - @DefaultValue(DEFAULT_DEGREE) long degree, - @QueryParam("capacity") - @DefaultValue(DEFAULT_CAPACITY) long capacity, - @QueryParam("limit") - @DefaultValue(DEFAULT_PATHS_LIMIT) long limit) { - LOG.debug("Graph [{}] get paths from '{}', to '{}' with " + - "direction {}, edge label {}, max depth '{}', " + - "max degree '{}', capacity '{}' and limit '{}'", - graph, source, target, direction, edgeLabel, depth, - degree, capacity, limit); - - Id sourceId = VertexAPI.checkAndParseVertexId(source); - Id targetId = VertexAPI.checkAndParseVertexId(target); - Directions dir = Directions.convert(EdgeAPI.parseDirection(direction)); - - HugeGraph g = graph(manager, graph); - PathsTraverser traverser = new PathsTraverser(g); - HugeTraverser.PathSet paths = traverser.paths(sourceId, dir, targetId, - dir.opposite(), edgeLabel, - depth, degree, capacity, - limit); - return manager.serializer(g).writePaths("paths", paths, false); - } - private static class Request { @JsonProperty("sources") From 924c5e174b626fba1be366c35c3713645cd28490 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Wed, 16 Sep 2020 11:53:38 +0800 Subject: [PATCH 09/20] move jaccard similar into jaccard similarity Change-Id: Ia6ff5efd7e2a195f85427992b7bff875d30dec48 --- .../api/traversers/JaccardSimilarAPI.java | 105 ------------------ .../api/traversers/JaccardSimilarityAPI.java | 64 ++++++++++- 2 files changed, 62 insertions(+), 107 deletions(-) delete mode 100644 hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java deleted file mode 100644 index 5fc54796b0..0000000000 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarAPI.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.api.traversers; - -import java.util.Map; - -import javax.inject.Singleton; -import javax.ws.rs.Consumes; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; - -import org.slf4j.Logger; - -import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.backend.id.Id; -import com.baidu.hugegraph.core.GraphManager; -import com.baidu.hugegraph.server.RestServer; -import com.baidu.hugegraph.structure.HugeVertex; -import com.baidu.hugegraph.traversal.algorithm.EdgeStep; -import com.baidu.hugegraph.traversal.algorithm.JaccardSimilarTraverser; -import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.Log; -import com.codahale.metrics.annotation.Timed; -import com.fasterxml.jackson.annotation.JsonProperty; - -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser - .DEFAULT_CAPACITY; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_LIMIT; - -@Path("graphs/{graph}/traversers/jaccardsimilar") -@Singleton -public class JaccardSimilarAPI extends TraverserAPI { - - private static final Logger LOG = Log.logger(RestServer.class); - - @POST - @Timed - @Consumes(APPLICATION_JSON) - @Produces(APPLICATION_JSON_WITH_CHARSET) - public String post(@Context GraphManager manager, - @PathParam("graph") String graph, - Request request) { - E.checkArgumentNotNull(request, "The request body can't be null"); - E.checkArgumentNotNull(request.vertex, - "The source vertex of request can't be null"); - E.checkArgument(request.step != null, - "The steps of request can't be null"); - E.checkArgument(request.top >= 0, - "The top must be >= 0, but got: %s", request.top); - - LOG.debug("Graph [{}] get jaccard similars from source vertex '{}', " + - "with step '{}', top '{}' and capacity '{}'", - graph, request.vertex, request.step, request.top); - - HugeGraph g = graph(manager, graph); - Id sourceId = HugeVertex.getIdValue(request.vertex); - - EdgeStep step = step(g, request.step); - - JaccardSimilarTraverser traverser = new JaccardSimilarTraverser(g); - Map results = traverser.jaccardSimilars(sourceId, step, - request.top, - request.capacity); - return manager.serializer(g).writeMap(results); - } - - private static class Request { - - @JsonProperty("vertex") - public Object vertex; - @JsonProperty("step") - public TraverserAPI.Step step; - @JsonProperty("top") - public int top = Integer.valueOf(DEFAULT_LIMIT); - @JsonProperty("capacity") - public long capacity = Long.valueOf(DEFAULT_CAPACITY); - - @Override - public String toString() { - return String.format("Request{vertex=%s,step=%s,top=%s," + - "capacity=%s}", this.vertex, this.step, - this.top, this.capacity); - } - } -} diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java index 5c2449ce8d..1ee52c8397 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java @@ -19,11 +19,17 @@ package com.baidu.hugegraph.api.traversers; +import java.util.Map; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_LIMIT; import javax.inject.Singleton; +import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; @@ -33,22 +39,26 @@ import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.api.API; import com.baidu.hugegraph.api.graph.EdgeAPI; import com.baidu.hugegraph.api.graph.VertexAPI; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; +import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; +import com.baidu.hugegraph.traversal.algorithm.JaccardSimilarTraverser; import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.JsonUtil; import com.baidu.hugegraph.util.Log; import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; @Path("graphs/{graph}/traversers/jaccardsimilarity") @Singleton -public class JaccardSimilarityAPI extends API { +public class JaccardSimilarityAPI extends TraverserAPI { private static final Logger LOG = Log.logger(RestServer.class); @@ -78,4 +88,54 @@ public String get(@Context GraphManager manager, return JsonUtil.toJson(ImmutableMap.of("jaccard_similarity", similarity)); } + + @POST + @Timed + @Consumes(APPLICATION_JSON) + @Produces(APPLICATION_JSON_WITH_CHARSET) + public String post(@Context GraphManager manager, + @PathParam("graph") String graph, + Request request) { + E.checkArgumentNotNull(request, "The request body can't be null"); + E.checkArgumentNotNull(request.vertex, + "The source vertex of request can't be null"); + E.checkArgument(request.step != null, + "The steps of request can't be null"); + E.checkArgument(request.top >= 0, + "The top must be >= 0, but got: %s", request.top); + + LOG.debug("Graph [{}] get jaccard similars from source vertex '{}', " + + "with step '{}', top '{}' and capacity '{}'", + graph, request.vertex, request.step, request.top); + + HugeGraph g = graph(manager, graph); + Id sourceId = HugeVertex.getIdValue(request.vertex); + + EdgeStep step = step(g, request.step); + + JaccardSimilarTraverser traverser = new JaccardSimilarTraverser(g); + Map results = traverser.jaccardSimilars(sourceId, step, + request.top, + request.capacity); + return manager.serializer(g).writeMap(results); + } + + private static class Request { + + @JsonProperty("vertex") + public Object vertex; + @JsonProperty("step") + public TraverserAPI.Step step; + @JsonProperty("top") + public int top = Integer.valueOf(DEFAULT_LIMIT); + @JsonProperty("capacity") + public long capacity = Long.valueOf(DEFAULT_CAPACITY); + + @Override + public String toString() { + return String.format("Request{vertex=%s,step=%s,top=%s," + + "capacity=%s}", this.vertex, this.step, + this.top, this.capacity); + } + } } From f31cf3adff9c7e1351744ce06a0cd057e96d3e23 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Wed, 16 Sep 2020 18:12:02 +0800 Subject: [PATCH 10/20] improve Change-Id: Ib43614c252912427c6ddaeb52c974a59b06934e6 --- .../hugegraph/api/traversers/JaccardSimilarityAPI.java | 3 ++- .../{TemplatePathAPI.java => TemplatePathsAPI.java} | 6 +++--- .../com/baidu/hugegraph/api/traversers/TraverserAPI.java | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) rename hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/{TemplatePathAPI.java => TemplatePathsAPI.java} (96%) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java index 1ee52c8397..8de41fa3c3 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java @@ -106,7 +106,8 @@ public String post(@Context GraphManager manager, LOG.debug("Graph [{}] get jaccard similars from source vertex '{}', " + "with step '{}', top '{}' and capacity '{}'", - graph, request.vertex, request.step, request.top); + graph, request.vertex, request.step, + request.top, request.capacity); HugeGraph g = graph(manager, graph); Id sourceId = HugeVertex.getIdValue(request.vertex); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathsAPI.java similarity index 96% rename from hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java rename to hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathsAPI.java index e7ab772787..c0e13d8fc0 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TemplatePathsAPI.java @@ -53,7 +53,7 @@ @Path("graphs/{graph}/traversers/templatepaths") @Singleton -public class TemplatePathAPI extends TraverserAPI { +public class TemplatePathsAPI extends TraverserAPI { private static final Logger LOG = Log.logger(RestServer.class); @@ -134,8 +134,8 @@ private static class Request { @Override public String toString() { - return String.format("PathRequest{sources=%s,targets=%s,steps=%s," + - "withRing=%s,capacity=%s,limit=%s," + + return String.format("TemplatePathsRequest{sources=%s,targets=%s," + + "steps=%s,withRing=%s,capacity=%s,limit=%s," + "withVertex=%s}", this.sources, this.targets, this.steps, this.withRing, this.capacity, this.limit, diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java index 1e130f0adf..c9af8595da 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/TraverserAPI.java @@ -39,7 +39,7 @@ protected static EdgeStep step(HugeGraph graph, Step step) { } protected static TemplatePathsTraverser.RepeatEdgeStep repeatEdgeStep( - HugeGraph graph, TemplatePathAPI.RepeatEdgeStep step) { + HugeGraph graph, TemplatePathsAPI.RepeatEdgeStep step) { return new TemplatePathsTraverser.RepeatEdgeStep(graph, step.direction, step.labels, step.properties, From b9706e1777e47efff757b9e71a132da3c3793c30 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Thu, 17 Sep 2020 18:48:31 +0800 Subject: [PATCH 11/20] improve Change-Id: I5ef4b33e8c5b4ade0b372d3810e596d335f37f29 --- .../algorithm/CollectionPathsTraverser.java | 130 ++++----- .../traversal/algorithm/PathsTraverser.java | 49 ++-- .../algorithm/TemplatePathsTraverser.java | 257 +++++++++++------- .../traversal/algorithm/TpTraverser.java | 22 +- 4 files changed, 271 insertions(+), 187 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java index 600ef5a654..926dd63b55 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -22,13 +22,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; import javax.ws.rs.core.MultivaluedMap; @@ -42,7 +40,6 @@ import com.baidu.hugegraph.structure.HugeVertex; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; -import com.google.common.collect.ImmutableList; public class CollectionPathsTraverser extends TpTraverser { @@ -85,24 +82,18 @@ public Collection paths(Iterator sources, this.singleTraverser(sourceList, targetList, step, nearest, capacity, limit); - Collection paths = new HashSet<>(); while (true) { if (--depth < 0 || traverser.reachLimit()) { break; } - Collection foundPaths = traverser.forward(); - paths.addAll(foundPaths); + traverser.forward(); if (--depth < 0 || traverser.reachLimit()) { break; } - foundPaths = traverser.backward(); - for (Path path : foundPaths) { - path.reverse(); - paths.add(path); - } + traverser.backward(); } - return paths; + return traverser.paths(); } private Traverser singleTraverser(List sources, List targets, @@ -132,12 +123,14 @@ public Traverser(EdgeStep step, long capacity, long limit) { this.limit = limit; } - public Collection forward() { - return ImmutableList.of(); + public void forward() { + } + + public void backward() { } - public Collection backward() { - return ImmutableList.of(); + public Set paths() { + return new PathSet(); } public int pathCount() { @@ -169,7 +162,7 @@ private class ConcurrentTraverser extends Traverser { private ConcurrentMultiValuedMap targetsAll = new ConcurrentMultiValuedMap<>(); - protected AtomicInteger pathCount; + private Set paths; public ConcurrentTraverser(Collection sources, Collection targets, EdgeStep step, @@ -183,14 +176,14 @@ public ConcurrentTraverser(Collection sources, } this.sourcesAll.putAll(this.sources); this.targetsAll.putAll(this.targets); - this.pathCount = new AtomicInteger(0); + + this.paths = ConcurrentHashMap.newKeySet(); } /** * Search forward from sources */ - public Collection forward() { - Set paths = ConcurrentHashMap.newKeySet(); + public void forward() { ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); // Traversal vertices of previous level @@ -214,8 +207,7 @@ public Collection forward() { for (Node node : this.targetsAll.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - this.pathCount.incrementAndGet(); + this.paths.add(new Path(target, path)); if (this.reachLimit()) { return; } @@ -232,16 +224,15 @@ public Collection forward() { // Re-init sources this.sources = newVertices; // Record all passed vertices - this.sourcesAll.putAll(newVertices); - - return paths; + for (Map.Entry> entry : newVertices.entrySet()) { + this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + } } /** * Search backward from target */ - public Set backward() { - Set paths = ConcurrentHashMap.newKeySet(); + public void backward() { ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); this.step.swithDirection(); @@ -267,8 +258,9 @@ public Set backward() { for (Node node : this.sourcesAll.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - this.pathCount.incrementAndGet(); + Path newPath = new Path(target, path); + newPath.reverse(); + this.paths.add(newPath); if (this.reachLimit()) { return; } @@ -286,14 +278,19 @@ public Set backward() { // Re-init targets this.targets = newVertices; // Record all passed vertices - this.targetsAll.putAll(newVertices); + for (Map.Entry> entry : newVertices.entrySet()) { + this.targetsAll.addAll(entry.getKey(), entry.getValue()); + } + } - return paths; + @Override + public Set paths() { + return this.paths; } @Override public int pathCount() { - return this.pathCount.get(); + return this.paths.size(); } protected int accessedNodes() { @@ -308,7 +305,7 @@ private class SingleAllTraverser extends Traverser { private MultivaluedMap sourcesAll = newMultivalueMap(); private MultivaluedMap targetsAll = newMultivalueMap(); - private int pathCount; + private PathSet paths; public SingleAllTraverser(Collection sources, Collection targets, @@ -322,14 +319,13 @@ public SingleAllTraverser(Collection sources, } this.sourcesAll.putAll(this.sources); this.targetsAll.putAll(this.targets); - this.pathCount = 0; + this.paths = new PathSet(); } /** * Search forward from sources */ - public PathSet forward() { - PathSet paths = new PathSet(); + public void forward() { MultivaluedMap newVertices = newMultivalueMap(); Iterator edges; // Traversal vertices of previous level @@ -352,10 +348,9 @@ public PathSet forward() { for (Node node : this.targetsAll.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.pathCount; + this.paths.add(new Path(target, path)); if (this.reachLimit()) { - return paths; + return; } } } @@ -370,15 +365,15 @@ public PathSet forward() { // Re-init targets this.sources = newVertices; // Record all passed vertices - this.sourcesAll.putAll(newVertices); - - return paths; + for (Map.Entry> entry : newVertices.entrySet()) { + this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + } } /** * Search backward from target */ - public PathSet backward() { + public void backward() { PathSet paths = new PathSet(); MultivaluedMap newVertices = newMultivalueMap(); this.step.swithDirection(); @@ -403,10 +398,11 @@ public PathSet backward() { for (Node node : this.sourcesAll.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.pathCount; + Path newPath = new Path(target, path); + newPath.reverse(); + this.paths.add(newPath); if (this.reachLimit()) { - return paths; + return; } } } @@ -422,14 +418,19 @@ public PathSet backward() { // Re-init targets this.targets = newVertices; // Record all passed vertices - this.targetsAll.putAll(newVertices); + for (Map.Entry> entry : newVertices.entrySet()) { + this.targetsAll.addAll(entry.getKey(), entry.getValue()); + } + } - return paths; + @Override + public Set paths() { + return this.paths; } @Override public int pathCount() { - return this.pathCount; + return this.paths.size(); } protected int accessedNodes() { @@ -445,7 +446,7 @@ private class SingleNearestTraverser extends Traverser { private Map sourcesAll = new HashMap<>(); private Map targetsAll = new HashMap<>(); - private int pathCount; + private PathSet paths; public SingleNearestTraverser(Collection sources, Collection targets, @@ -460,16 +461,15 @@ public SingleNearestTraverser(Collection sources, } this.sourcesAll.putAll(this.sources); this.targetsAll.putAll(this.targets); - this.pathCount = 0; + this.paths = new PathSet(); } /** * Search forward from sources */ - public PathSet forward() { + public void forward() { LOG.info("Forward with sources size {} and sources all size {}", this.sources.size(), this.sourcesAll.size()); - PathSet paths = new PathSet(); Map newVertices = new HashMap<>(); Iterator edges; // Traversal vertices of previous level @@ -492,10 +492,9 @@ public PathSet forward() { Node node = this.targetsAll.get(target); List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.pathCount; + this.paths.add(new Path(target, path)); if (this.reachLimit()) { - return paths; + return; } } } @@ -514,16 +513,14 @@ public PathSet forward() { } LOG.info("Done forward with sources size {} and sources all size {}", this.sources.size(), this.sourcesAll.size()); - return paths; } /** * Search backward from target */ - public PathSet backward() { + public void backward() { LOG.info("Backward with targets size {} and targets all size {}", this.targets.size(), this.targetsAll.size()); - PathSet paths = new PathSet(); Map newVertices = new HashMap<>(); this.step.swithDirection(); Iterator edges; @@ -547,10 +544,11 @@ public PathSet backward() { Node node = this.sourcesAll.get(target); List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.pathCount; + Path newPath = new Path(target, path); + newPath.reverse(); + this.paths.add(newPath); if (this.reachLimit()) { - return paths; + return; } } } @@ -570,12 +568,16 @@ public PathSet backward() { } LOG.info("Done backward with sources size {} and sources all size {}", this.targets.size(), this.targetsAll.size()); - return paths; + } + + @Override + public Set paths() { + return this.paths; } @Override public int pathCount() { - return this.pathCount; + return this.paths.size(); } protected int accessedNodes() { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathsTraverser.java index cda75814f5..9b4955d001 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathsTraverser.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -71,18 +70,14 @@ public PathSet paths(Id sourceV, Directions sourceDir, if (--depth < 0 || traverser.reachLimit()) { break; } - List foundPaths = traverser.forward(sourceDir); - paths.addAll(foundPaths); + traverser.forward(sourceDir); if (--depth < 0 || traverser.reachLimit()) { break; } - foundPaths = traverser.backward(targetDir); - for (Path path : foundPaths) { - path.reverse(); - paths.add(path); - } + traverser.backward(targetDir); } + paths.addAll(traverser.paths()); return paths; } @@ -97,7 +92,7 @@ private class Traverser { private final long degree; private final long capacity; private final long limit; - private long count; + private PathSet paths; public Traverser(Id sourceV, Id targetV, Id label, long degree, long capacity, long limit) { @@ -109,14 +104,13 @@ public Traverser(Id sourceV, Id targetV, Id label, this.degree = degree; this.capacity = capacity; this.limit = limit; - this.count = 0L; + this.paths = new PathSet(); } /** * Search forward from source */ - public List forward(Directions direction) { - List paths = new ArrayList<>(); + public void forward(Directions direction) { MultivaluedMap newVertices = newMultivalueMap(); Iterator edges; // Traversal vertices of previous level @@ -139,10 +133,9 @@ public List forward(Directions direction) { for (Node node : this.targetsAll.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.count; + this.paths.add(new Path(target, path)); if (this.reachLimit()) { - return paths; + return; } } } @@ -156,16 +149,15 @@ public List forward(Directions direction) { // Re-init sources this.sources = newVertices; // Record all passed vertices - this.sourcesAll.putAll(newVertices); - - return paths; + for (Map.Entry> entry : newVertices.entrySet()) { + this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + } } /** * Search backward from target */ - public List backward(Directions direction) { - List paths = new ArrayList<>(); + public void backward(Directions direction) { MultivaluedMap newVertices = newMultivalueMap(); Iterator edges; // Traversal vertices of previous level @@ -188,10 +180,11 @@ public List backward(Directions direction) { for (Node node : this.sourcesAll.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.count; + Path newPath = new Path(target, path); + newPath.reverse(); + this.paths.add(newPath); if (this.reachLimit()) { - return paths; + return; } } } @@ -206,9 +199,13 @@ public List backward(Directions direction) { // Re-init targets this.targets = newVertices; // Record all passed vertices - this.targetsAll.putAll(newVertices); + for (Map.Entry> entry : newVertices.entrySet()) { + this.targetsAll.addAll(entry.getKey(), entry.getValue()); + } + } - return paths; + public PathSet paths() { + return this.paths; } private int accessedNodes() { @@ -217,7 +214,7 @@ private int accessedNodes() { private boolean reachLimit() { checkCapacity(this.capacity, this.accessedNodes(), "paths"); - if (this.limit == NO_LIMIT || this.count < this.limit) { + if (this.limit == NO_LIMIT || this.paths.size() < this.limit) { return false; } return true; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index d6f86d9d0e..ef51ccab5a 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -22,13 +22,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; import javax.ws.rs.core.MultivaluedMap; @@ -86,24 +84,23 @@ public Set templatePaths(Iterator sources, } Traverser traverser = totalSteps >= this.concurrentDepth() ? new ConcurrentTraverser(sourceList, targetList, - steps, capacity, limit) : + steps, withRing, + capacity, limit) : new SingleTraverser(sourceList, targetList, - steps, withRing, capacity, limit); - Set paths = new HashSet<>(); + steps, withRing, + capacity, limit); + do { // Forward - paths.addAll(traverser.forward()); + traverser.forward(); if (traverser.finish()) { - return paths; + return traverser.paths(); } // Backward - for (Path path : traverser.backward()) { - path.reverse(); - paths.add(path); - } + traverser.backward(); if (traverser.finish()) { - return paths; + return traverser.paths(); } } while (true); } @@ -115,24 +112,35 @@ private class Traverser { protected final long capacity; protected final long limit; protected int totalSteps; + protected boolean withRing; + protected int sourceIndex; + protected int targetIndex; + + protected boolean sourceFinishOneStep = false; + protected boolean targetFinishOneStep = false; public Traverser(List steps, - long capacity, long limit) { + long capacity, long limit, boolean withRing) { this.steps = steps; this.capacity = capacity; this.limit = limit; + this.withRing = withRing; this.stepCount = 0; for (RepeatEdgeStep step : steps) { this.totalSteps += step.maxTimes; } + this.sourceIndex = 0; + this.targetIndex = this.steps.size() - 1; } - public Set forward() { - return new PathSet(); + public void forward() { } - public Set backward() { + public void backward() { + } + + public Set paths() { return new PathSet(); } @@ -141,7 +149,7 @@ public int pathCount() { } protected boolean finish() { - return this.stepCount == this.totalSteps; + return this.stepCount >= this.totalSteps || this.reachLimit(); } protected boolean lastStep() { @@ -160,6 +168,11 @@ protected boolean reachLimit() { } return true; } + + public boolean lastSuperStep() { + return this.targetIndex == this.sourceIndex || + this.targetIndex == this.sourceIndex + 1; + } } private class ConcurrentTraverser extends Traverser { @@ -168,55 +181,78 @@ private class ConcurrentTraverser extends Traverser { new ConcurrentMultiValuedMap<>(); private ConcurrentMultiValuedMap targets = new ConcurrentMultiValuedMap<>(); + private ConcurrentMultiValuedMap sourcesAll = + new ConcurrentMultiValuedMap<>(); + private ConcurrentMultiValuedMap targetsAll = + new ConcurrentMultiValuedMap<>(); - protected AtomicInteger pathCount; + private Set paths; public ConcurrentTraverser(Collection sources, Collection targets, - List steps, + List steps, boolean withRing, long capacity, long limit) { - super(steps, capacity, limit); + super(steps, capacity, limit, withRing); for (Id id : sources) { this.sources.add(id, new Node(id)); } for (Id id : targets) { this.targets.add(id, new Node(id)); } - this.pathCount = new AtomicInteger(0); + this.paths = ConcurrentHashMap.newKeySet(); } /** * Search forward from sources */ - public Set forward() { - Set paths = ConcurrentHashMap.newKeySet(); + public void forward() { ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); - RepeatEdgeStep step = this.steps.get(this.stepCount / 2); + RepeatEdgeStep currentStep = null; + // Find next step to forward + for (int i = 0; i < this.steps.size(); i++) { + RepeatEdgeStep step = this.steps.get(i); + if (step.remainTimes() > 0) { + currentStep = step; + this.sourceIndex = i; + break; + } + } + if (currentStep == null) { + return; + } + + // Re-init source all if last forward finished one super step and + // not last super step + if (this.sourceFinishOneStep && !this.lastSuperStep()) { + this.sourcesAll = new ConcurrentMultiValuedMap<>(); + this.sourceFinishOneStep = false; + } + // Traversal vertices of previous level + RepeatEdgeStep finalCurrentStep = currentStep; traverseIds(this.sources.keySet().iterator(), vid -> { if (this.reachLimit()) { return; } - Iterator edges = edgesOfVertex(vid, step); + Iterator edges = edgesOfVertex(vid, finalCurrentStep); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); Id target = edge.id().otherVertexId(); for (Node n : this.sources.get(vid)) { // If have loop, skip target - if (n.contains(target)) { + if (!this.withRing && n.contains(target)) { continue; } // If cross point exists, path found, concat them - if (this.lastStep() && + if (this.lastSuperStep() && this.targets.containsKey(target)) { for (Node node : this.targets.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - this.pathCount.incrementAndGet(); + this.paths.add(new Path(target, path)); if (this.reachLimit()) { return; } @@ -230,30 +266,57 @@ public Set forward() { } }); - // Re-init sources - this.sources = newVertices; + for (Map.Entry> entry : newVertices.entrySet()) { + this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + } + currentStep.decreaseTimes(); + // Re-init sources + if (currentStep.remainTimes() > 0) { + this.sources = newVertices; + } else { + this.sources = this.sourcesAll; + this.sourceFinishOneStep = true; + } this.stepCount++; - - return paths; } /** * Search backward from target */ - public Set backward() { - Set paths = ConcurrentHashMap.newKeySet(); + public void backward() { ConcurrentMultiValuedMap newVertices = new ConcurrentMultiValuedMap<>(); - int index = this.steps.size() - stepCount / 2 - 1; - RepeatEdgeStep step = this.steps.get(index); - step.swithDirection(); + RepeatEdgeStep currentStep = null; + + // Find next step to backward + for (int i = this.steps.size() - 1; i >= 0; i--) { + RepeatEdgeStep step = this.steps.get(i); + if (step.remainTimes() > 0) { + currentStep = step; + this.targetIndex = i; + break; + } + } + if (currentStep == null) { + return; + } + + // Re-init target all if last forward finished one super step and + // not last super step + if (this.targetFinishOneStep && !this.lastSuperStep()) { + this.targetsAll = new ConcurrentMultiValuedMap<>(); + this.targetFinishOneStep = false; + } + + currentStep.swithDirection(); // Traversal vertices of previous level + RepeatEdgeStep finalCurrentStep = currentStep; traverseIds(this.targets.keySet().iterator(), vid -> { if (this.reachLimit()) { return; } - Iterator edges = edgesOfVertex(vid, step); + Iterator edges = edgesOfVertex(vid, finalCurrentStep); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); @@ -271,8 +334,9 @@ public Set backward() { for (Node node : this.sources.get(target)) { List path = n.joinPath(node); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - this.pathCount.incrementAndGet(); + Path newPath = new Path(target, path); + newPath.reverse(); + this.paths.add(newPath); if (this.reachLimit()) { return; } @@ -286,21 +350,42 @@ public Set backward() { } }); - // Re-init targets - this.targets = newVertices; + currentStep.swithDirection(); + for (Map.Entry> entry : newVertices.entrySet()) { + this.targetsAll.addAll(entry.getKey(), entry.getValue()); + } + + currentStep.decreaseTimes(); + // Re-init targets + if (currentStep.remainTimes() > 0) { + this.targets = newVertices; + } else { + this.targets = this.targetsAll; + this.targetFinishOneStep = true; + } this.stepCount++; + } - return paths; + @Override + public Set paths() { + return this.paths; } @Override public int pathCount() { - return this.pathCount.get(); + return this.paths.size(); } protected int accessedNodes() { - return this.sources.size() + this.targets.size(); + int size = 0; + for (List value : this.sourcesAll.values()) { + size += value.size(); + } + for (List value : this.targetsAll.values()) { + size += value.size(); + } + return size; } } @@ -311,41 +396,25 @@ private class SingleTraverser extends Traverser { private MultivaluedMap sourcesAll = newMultivalueMap(); private MultivaluedMap targetsAll = newMultivalueMap(); - private int sourceIndex; - private int targetIndex; - private boolean sourceFinishOneStep = false; - private boolean targetFinishOneStep = false; - private int pathCount; - private boolean withRing; + private Set paths; public SingleTraverser(Collection sources, Collection targets, List steps, boolean withRing, long capacity, long limit) { - this(sources, targets, steps, capacity, limit); - this.withRing = withRing; - - } - - public SingleTraverser(Collection sources, Collection targets, - List steps, long capacity, - long limit) { - super(steps, capacity, limit); + super(steps, capacity, limit, withRing); for (Id id : sources) { this.sources.add(id, new Node(id)); } for (Id id : targets) { this.targets.add(id, new Node(id)); } - this.sourceIndex = 0; - this.targetIndex = this.steps.size() - 1; - this.pathCount = 0; + this.paths = new PathSet(); } /** * Search forward from sources */ - public PathSet forward() { - PathSet paths = new PathSet(); + public void forward() { MultivaluedMap newVertices = newMultivalueMap(); RepeatEdgeStep currentStep = null; // Find next step to forward @@ -358,7 +427,7 @@ public PathSet forward() { } } if (currentStep == null) { - return new PathSet(); + return; } // Re-init source all if last forward finished one super step and @@ -380,7 +449,7 @@ public PathSet forward() { for (Node n : entry.getValue()) { // If have loop, skip target - if (n.contains(target)) { + if (!this.withRing && n.contains(target)) { continue; } @@ -390,10 +459,9 @@ public PathSet forward() { for (Node node : this.targetsAll.get(target)) { List path = joinPath(n, node, withRing); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.pathCount; + this.paths.add(new Path(target, path)); if (this.reachLimit()) { - return paths; + return; } } } @@ -405,7 +473,9 @@ public PathSet forward() { } } - this.sourcesAll.putAll(newVertices); + for (Map.Entry> entry : newVertices.entrySet()) { + this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + } currentStep.decreaseTimes(); // Re-init sources @@ -415,17 +485,13 @@ public PathSet forward() { this.sources = this.sourcesAll; this.sourceFinishOneStep = true; } - this.stepCount++; - - return paths; } /** * Search backward from target */ - public PathSet backward() { - PathSet paths = new PathSet(); + public void backward() { MultivaluedMap newVertices = newMultivalueMap(); RepeatEdgeStep currentStep = null; @@ -439,7 +505,7 @@ public PathSet backward() { } } if (currentStep == null) { - return new PathSet(); + return; } // Re-init target all if last forward finished one super step and @@ -462,7 +528,7 @@ public PathSet backward() { for (Node n : entry.getValue()) { // If have loop, skip target - if (n.contains(target)) { + if (!this.withRing && n.contains(target)) { continue; } @@ -472,10 +538,11 @@ public PathSet backward() { for (Node node : this.sourcesAll.get(target)) { List path = joinPath(n, node, withRing); if (!path.isEmpty()) { - paths.add(new Path(target, path)); - ++this.pathCount; + Path newPath = new Path(target, path); + newPath.reverse(); + this.paths.add(newPath); if (this.reachLimit()) { - return paths; + return; } } } @@ -489,7 +556,9 @@ public PathSet backward() { currentStep.swithDirection(); - this.targetsAll.putAll(newVertices); + for (Map.Entry> entry : newVertices.entrySet()) { + this.targetsAll.addAll(entry.getKey(), entry.getValue()); + } currentStep.decreaseTimes(); // Re-init targets @@ -499,24 +568,28 @@ public PathSet backward() { this.targets = this.targetsAll; this.targetFinishOneStep = true; } - this.stepCount++; + } - return paths; + @Override + public Set paths() { + return this.paths; } @Override public int pathCount() { - return this.pathCount; + return this.paths.size(); } protected int accessedNodes() { - return this.sources.size() + this.targets.size(); - } - - public boolean lastSuperStep() { - return this.targetIndex == this.sourceIndex || - this.targetIndex == this.sourceIndex + 1; + int size = 0; + for (List value : this.sourcesAll.values()) { + size += value.size(); + } + for (List value : this.targetsAll.values()) { + size += value.size(); + } + return size; } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index 1772c2d49e..5668cf39ee 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -20,8 +20,10 @@ package com.baidu.hugegraph.traversal.algorithm; import java.util.Iterator; +import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; @@ -154,22 +156,32 @@ protected boolean match(Element elem, String key, Object value) { } public class ConcurrentMultiValuedMap - extends ConcurrentHashMap> { + extends ConcurrentHashMap> { public ConcurrentMultiValuedMap() { super(); } public void add(K key, V value) { - Set values = this.get(key); + List values = this.getValues(key); + values.add(value); + } + + public void addAll(K key, List value) { + List values = this.getValues(key); + values.addAll(value); + } + + public List getValues(K key) { + List values = this.get(key); if (values == null) { - values = ConcurrentHashMap.newKeySet(); - Set old = this.putIfAbsent(key, values); + values = new CopyOnWriteArrayList<>(); + List old = this.putIfAbsent(key, values); if (old != null) { values = old; } } - values.add(value); + return values; } } } From ebe3157ef79189cc661c1a93cfbcb270a11d0750 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Mon, 21 Sep 2020 17:35:23 +0800 Subject: [PATCH 12/20] fix kout/kneighbor withVertex not work Change-Id: Icbc65aa24e069b67bd5653b79473da48e2f973c6 --- .../java/com/baidu/hugegraph/api/traversers/KneighborAPI.java | 3 +++ .../main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java | 3 +++ .../main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java | 2 +- 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java index fff1630409..58cddfca31 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java @@ -144,6 +144,9 @@ public String post(@Context GraphManager manager, Iterator iter = QueryResults.emptyIterator(); if (request.withVertex) { Set ids = new HashSet<>(); + for (HugeTraverser.Node node : results) { + ids.add(node.id()); + } for (HugeTraverser.Path p : paths) { ids.addAll(p.vertices()); } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java index 5031758d86..dd341c87ad 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java @@ -148,6 +148,9 @@ public String post(@Context GraphManager manager, Iterator iter = QueryResults.emptyIterator(); if (request.withVertex) { Set ids = new HashSet<>(); + for (Node node : results) { + ids.add(node.id()); + } for (HugeTraverser.Path p : paths) { ids.addAll(p.vertices()); } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java index f06a867c4d..27ea55b2eb 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PathsAPI.java @@ -162,7 +162,7 @@ private static class Request { @JsonProperty("max_depth") public int depth; @JsonProperty("nearest") - public boolean nearest = true; + public boolean nearest = false; @JsonProperty("capacity") public long capacity = Long.valueOf(DEFAULT_CAPACITY); @JsonProperty("limit") From 25cb47ad2f8ad21d0657cd96277f79ec37c981a2 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Mon, 21 Sep 2020 18:04:17 +0800 Subject: [PATCH 13/20] rebase ramtable Change-Id: I606772946a2a55f7472274242fb18f4ca91c189a --- .../hugegraph/job/algorithm/Consumers.java | 205 ------------------ .../traversal/algorithm/TpTraverser.java | 11 +- 2 files changed, 8 insertions(+), 208 deletions(-) delete mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java deleted file mode 100644 index 1c68413fc0..0000000000 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/job/algorithm/Consumers.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * 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.job.algorithm; - -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -import org.slf4j.Logger; - -import com.baidu.hugegraph.HugeException; -import com.baidu.hugegraph.task.TaskManager.ContextCallable; -import com.baidu.hugegraph.util.ExecutorUtil; -import com.baidu.hugegraph.util.Log; - -public class Consumers { - - public static final int CPUS = Runtime.getRuntime().availableProcessors(); - public static final int THREADS = 4 + CPUS / 4; - public static final int QUEUE_WORKER_SIZE = 1000; - - private static final Logger LOG = Log.logger(Consumers.class); - - private final ExecutorService executor; - private final Consumer consumer; - private final Runnable done; - - private final int workers; - private final int queueSize; - private final CountDownLatch latch; - private final BlockingQueue queue; - - private volatile boolean ending = false; - private volatile Throwable exception = null; - - public Consumers(ExecutorService executor, Consumer consumer) { - this(executor, consumer, null); - } - - public Consumers(ExecutorService executor, - Consumer consumer, Runnable done) { - this.executor = executor; - this.consumer = consumer; - this.done = done; - - int workers = THREADS; - if (this.executor instanceof ThreadPoolExecutor) { - workers = ((ThreadPoolExecutor) this.executor).getCorePoolSize(); - } - this.workers = workers; - this.queueSize = QUEUE_WORKER_SIZE * workers; - this.latch = new CountDownLatch(workers); - this.queue = new ArrayBlockingQueue<>(this.queueSize); - } - - public void start(String name) { - this.ending = false; - this.exception = null; - if (this.executor == null) { - return; - } - LOG.info("Starting {} workers[{}] with queue size {}...", - this.workers, name, this.queueSize); - for (int i = 0; i < this.workers; i++) { - this.executor.submit(new ContextCallable<>(this::runAndDone)); - } - } - - private Void runAndDone() { - try { - this.run(); - this.done(); - } catch (Throwable e) { - // Only the first exception of one thread can be stored - this.exception = e; - if (!(e instanceof StopExecution)) { - LOG.error("Error when running task", e); - } - this.done(); - } finally { - this.latch.countDown(); - } - return null; - } - - private void run() { - LOG.debug("Start to work..."); - while (!this.ending) { - this.consume(); - } - assert this.ending; - while (this.consume()); - - LOG.debug("Worker finished"); - } - - private boolean consume() { - V elem; - try { - elem = this.queue.poll(1, TimeUnit.SECONDS); - } catch (InterruptedException e) { - // ignore - return true; - } - if (elem == null) { - return false; - } - // do job - this.consumer.accept(elem); - return true; - } - - private void done() { - if (this.done != null) { - this.done.run(); - } - } - - public void provide(V v) throws Throwable { - if (this.executor == null) { - assert this.exception == null; - // do job directly if without thread pool - this.consumer.accept(v); - } else if (this.exception != null) { - throw this.exception; - } else { - try { - this.queue.put(v); - } catch (InterruptedException e) { - LOG.warn("Interrupted", e);; - } - } - } - - public void await() { - this.ending = true; - if (this.executor == null) { - // call done() directly if without thread pool - this.done(); - } else { - try { - this.latch.await(); - } catch (InterruptedException e) { - LOG.warn("Interrupted", e); - } - } - } - - public static ExecutorService newThreadPool(String prefix, int workers) { - if (workers == 0) { - return null; - } else { - if (workers < 0) { - assert workers == -1; - workers = Consumers.THREADS; - } else if (workers > Consumers.CPUS * 2) { - workers = Consumers.CPUS * 2; - } - String name = prefix + "-worker-%d"; - return ExecutorUtil.newFixedThreadPool(workers, name); - } - } - - public static RuntimeException wrapException(Throwable e) { - if (e instanceof RuntimeException) { - throw (RuntimeException) e; - } - throw new HugeException("Error when running task: %s", - HugeException.rootCause(e).getMessage(), e); - } - - public static class StopExecution extends HugeException { - - private static final long serialVersionUID = -371829356182454517L; - - public StopExecution(String message) { - super(message); - } - - public StopExecution(String message, Object... args) { - super(message, args); - } - } -} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index 5668cf39ee..7b630d1ba3 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -40,8 +40,8 @@ import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.iterator.FilterIterator; -import com.baidu.hugegraph.job.algorithm.Consumers; import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.util.Consumers; import jersey.repackaged.com.google.common.base.Objects; @@ -134,8 +134,13 @@ protected long traverse(Iterator iterator, Consumer consumer, } catch (Throwable e) { throw Consumers.wrapException(e); } finally { - consumers.await(); - CloseableIterator.closeIterator(iterator); + try { + consumers.await(); + } catch (Throwable e) { + Consumers.wrapException(e); + } finally { + CloseableIterator.closeIterator(iterator); + } } return total; } From bff5a6a67a3c1f0868bf2a23aefa3cecf13fb12e Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Tue, 22 Sep 2020 11:09:25 +0800 Subject: [PATCH 14/20] improve Change-Id: I4ab46bc68644d35fab1fa966e27a0d7ab87b433b --- .../baidu/hugegraph/api/traversers/AllShortestPathsAPI.java | 3 ++- .../hugegraph/api/traversers/WeightedShortestPathAPI.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java index 62859abdc4..941042893e 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/AllShortestPathsAPI.java @@ -84,7 +84,8 @@ public String get(@Context GraphManager manager, HugeGraph g = graph(manager, graph); ShortestPathTraverser traverser = new ShortestPathTraverser(g); - List edgeLabels = ImmutableList.of(edgeLabel); + List edgeLabels = edgeLabel == null ? ImmutableList.of() : + ImmutableList.of(edgeLabel); HugeTraverser.PathSet paths = traverser.allShortestPaths( sourceId, targetId, dir, edgeLabels, depth, degree, skipDegree, capacity); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/WeightedShortestPathAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/WeightedShortestPathAPI.java index 9935b50876..bcebc05ff6 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/WeightedShortestPathAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/WeightedShortestPathAPI.java @@ -87,7 +87,8 @@ public String get(@Context GraphManager manager, E.checkArgumentNotNull(weight, "The weight property can't be null"); HugeGraph g = graph(manager, graph); - SingleSourceShortestPathTraverser traverser = new SingleSourceShortestPathTraverser(g); + SingleSourceShortestPathTraverser traverser = + new SingleSourceShortestPathTraverser(g); NodeWithWeight path = traverser.weightedShortestPath( sourceId, targetId, dir, edgeLabel, weight, From 8523889ee44e23f0bccc1282bd3ab2cbac3576f6 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Mon, 28 Sep 2020 14:56:34 +0800 Subject: [PATCH 15/20] extract common traverser from template paths traversers Change-Id: If6f0c1370a2376ef176ce2db39e002ae11590e8d --- .../traversal/algorithm/HugeTraverser.java | 36 +- .../algorithm/TemplatePathsTraverser.java | 643 +++++++----------- 2 files changed, 285 insertions(+), 394 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java index 7f823b0f91..a2083a6661 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java @@ -551,6 +551,26 @@ protected static MultivaluedMap newMultivalueMap() { return new MultivaluedHashMap<>(); } + protected static List joinPath(Node pre, Node back, boolean ring) { + // Get self path + List path = pre.path(); + + // Get reversed other path + List backPath = back.path(); + Collections.reverse(backPath); + + if (!ring) { + // Avoid loop in path + if (CollectionUtils.containsAny(path, backPath)) { + return ImmutableList.of(); + } + } + + // Append other path behind self path + path.addAll(backPath); + return path; + } + public static class Node { private Id id; @@ -586,21 +606,7 @@ public List path() { } public List joinPath(Node back) { - // Get self path - List path = this.path(); - - // Get reversed other path - List backPath = back.path(); - Collections.reverse(backPath); - - // Avoid loop in path - if (CollectionUtils.containsAny(path, backPath)) { - return ImmutableList.of(); - } - - // Append other path behind self path - path.addAll(backPath); - return path; + return HugeTraverser.joinPath(this, back, false); } public boolean contains(Id id) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index ef51ccab5a..b7183282ac 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -21,16 +21,15 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.BiConsumer; import javax.ws.rs.core.MultivaluedMap; -import org.apache.commons.collections.CollectionUtils; import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Vertex; import org.slf4j.Logger; @@ -42,7 +41,6 @@ import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; -import com.google.common.collect.ImmutableList; public class TemplatePathsTraverser extends TpTraverser { @@ -105,7 +103,7 @@ public Set templatePaths(Iterator sources, } while (true); } - private class Traverser { + private abstract class Traverser { protected final List steps; protected int stepCount; @@ -113,14 +111,25 @@ private class Traverser { protected final long limit; protected int totalSteps; protected boolean withRing; + protected int sourceIndex; protected int targetIndex; protected boolean sourceFinishOneStep = false; protected boolean targetFinishOneStep = false; - public Traverser(List steps, - long capacity, long limit, boolean withRing) { + protected Map> sources = this.newMultiValueMap(); + protected Map> sourcesAll = this.newMultiValueMap(); + protected Map> targets = this.newMultiValueMap(); + protected Map> targetsAll = this.newMultiValueMap(); + + protected Map> newVertices; + + private Set paths; + + public Traverser(Collection sources, Collection targets, + List steps, boolean withRing, + long capacity, long limit) { this.steps = steps; this.capacity = capacity; this.limit = limit; @@ -132,165 +141,199 @@ public Traverser(List steps, } this.sourceIndex = 0; this.targetIndex = this.steps.size() - 1; + + for (Id id : sources) { + this.addNode(this.sources, id, new Node(id)); + } + for (Id id : targets) { + this.addNode(this.targets, id, new Node(id)); + } + this.sourcesAll.putAll(this.sources); + this.targetsAll.putAll(this.targets); + + this.paths = this.newPathSet(); } public void forward() { + RepeatEdgeStep currentStep = this.step(true); + if (currentStep == null) { + return; + } + + this.beforeTraverse(true); + + // Traversal vertices of previous level + traverseOneLayer(this.sources, currentStep, this::forward); + + this.afterTraverse(currentStep, true); } public void backward() { - } + RepeatEdgeStep currentStep = this.step(false); + if (currentStep == null) { + return; + } - public Set paths() { - return new PathSet(); - } + this.beforeTraverse(false); - public int pathCount() { - return 0; - } + currentStep.swithDirection(); + // Traversal vertices of previous level + traverseOneLayer(this.targets, currentStep, this::backward); + currentStep.swithDirection(); - protected boolean finish() { - return this.stepCount >= this.totalSteps || this.reachLimit(); + this.afterTraverse(currentStep, false); } - protected boolean lastStep() { - return this.stepCount == this.totalSteps - 1; + public RepeatEdgeStep step(boolean forward) { + return forward ? this.forwardStep() : this.backwardStep(); } - protected int accessedNodes() { - return 0; + public void beforeTraverse(boolean forward) { + this.clearNewVertices(); + this.reInitAllIfNeeded(forward); } - protected boolean reachLimit() { - checkCapacity(this.capacity, this.accessedNodes(), - "template paths"); - if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { - return false; - } - return true; + public abstract void traverseOneLayer( + Map> vertices, + RepeatEdgeStep step, + BiConsumer consumer); + + public void afterTraverse(RepeatEdgeStep step, boolean forward) { + Map> all = forward ? this.sourcesAll : + this.targetsAll; + this.addNewVerticesToAll(all); + this.reInitCurrentIfNeeded(step, forward); + this.stepCount++; } - public boolean lastSuperStep() { - return this.targetIndex == this.sourceIndex || - this.targetIndex == this.sourceIndex + 1; + private void forward(Id v, RepeatEdgeStep step) { + this.traverseOne(v, step, true); } - } - private class ConcurrentTraverser extends Traverser { + private void backward(Id v, RepeatEdgeStep step) { + this.traverseOne(v, step, false); + } - private ConcurrentMultiValuedMap sources = - new ConcurrentMultiValuedMap<>(); - private ConcurrentMultiValuedMap targets = - new ConcurrentMultiValuedMap<>(); - private ConcurrentMultiValuedMap sourcesAll = - new ConcurrentMultiValuedMap<>(); - private ConcurrentMultiValuedMap targetsAll = - new ConcurrentMultiValuedMap<>(); + private void traverseOne(Id v, RepeatEdgeStep step, boolean forward) { + if (this.reachLimit()) { + return; + } - private Set paths; + Iterator edges = edgesOfVertex(v, step); + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); - public ConcurrentTraverser(Collection sources, - Collection targets, - List steps, boolean withRing, - long capacity, long limit) { - super(steps, capacity, limit, withRing); - for (Id id : sources) { - this.sources.add(id, new Node(id)); + this.processOne(v, target, forward); } - for (Id id : targets) { - this.targets.add(id, new Node(id)); + } + + private void processOne(Id source, Id target, boolean forward) { + if (forward) { + processOneForForward(source, target); + } else { + processOneForBackward(source, target); } - this.paths = ConcurrentHashMap.newKeySet(); } - /** - * Search forward from sources - */ - public void forward() { - ConcurrentMultiValuedMap newVertices = - new ConcurrentMultiValuedMap<>(); - RepeatEdgeStep currentStep = null; - // Find next step to forward - for (int i = 0; i < this.steps.size(); i++) { - RepeatEdgeStep step = this.steps.get(i); - if (step.remainTimes() > 0) { - currentStep = step; - this.sourceIndex = i; - break; + private void processOneForForward(Id source, Id target) { + for (Node n : this.sources.get(source)) { + // If have loop, skip target + if (!this.withRing && n.contains(target)) { + continue; + } + + // If cross point exists, path found, concat them + if (this.lastSuperStep() && + this.targetsAll.containsKey(target)) { + for (Node node : this.targetsAll.get(target)) { + List path = joinPath(n, node, this.withRing); + if (!path.isEmpty()) { + this.paths.add(new Path(target, path)); + if (this.reachLimit()) { + return; + } + } + } } - } - if (currentStep == null) { - return; - } - // Re-init source all if last forward finished one super step and - // not last super step - if (this.sourceFinishOneStep && !this.lastSuperStep()) { - this.sourcesAll = new ConcurrentMultiValuedMap<>(); - this.sourceFinishOneStep = false; + // Add node to next start-nodes + this.addNodeToNewVertices(target, new Node(target, n)); } + } - // Traversal vertices of previous level - RepeatEdgeStep finalCurrentStep = currentStep; - traverseIds(this.sources.keySet().iterator(), vid -> { - if (this.reachLimit()) { - return; + private void processOneForBackward(Id source, Id target) { + for (Node n : this.targets.get(source)) { + // If have loop, skip target + if (!this.withRing && n.contains(target)) { + continue; } - Iterator edges = edgesOfVertex(vid, finalCurrentStep); - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - for (Node n : this.sources.get(vid)) { - // If have loop, skip target - if (!this.withRing && n.contains(target)) { - continue; - } - // If cross point exists, path found, concat them - if (this.lastSuperStep() && - this.targets.containsKey(target)) { - for (Node node : this.targets.get(target)) { - List path = n.joinPath(node); - if (!path.isEmpty()) { - this.paths.add(new Path(target, path)); - if (this.reachLimit()) { - return; - } - } + // If cross point exists, path found, concat them + if (this.lastSuperStep() && + this.sourcesAll.containsKey(target)) { + for (Node node : this.sourcesAll.get(target)) { + List path = joinPath(n, node, this.withRing); + if (!path.isEmpty()) { + Path newPath = new Path(target, path); + newPath.reverse(); + this.paths.add(newPath); + if (this.reachLimit()) { + return; } } - - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); } } - }); - for (Map.Entry> entry : newVertices.entrySet()) { - this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + // Add node to next start-nodes + this.addNodeToNewVertices(target, new Node(target, n)); + } + } + + private void reInitAllIfNeeded(boolean forward) { + if (forward) { + // Re-init source all if last forward finished one super step + // and not last super step + if (this.sourceFinishOneStep && !this.lastSuperStep()) { + this.sourcesAll = this.newMultiValueMap(); + this.sourceFinishOneStep = false; + } + } else { + // Re-init target all if last forward finished one super step + // and not last super step + if (this.targetFinishOneStep && !this.lastSuperStep()) { + this.targetsAll = this.newMultiValueMap(); + this.targetFinishOneStep = false; + } } + } - currentStep.decreaseTimes(); - // Re-init sources - if (currentStep.remainTimes() > 0) { - this.sources = newVertices; + private void reInitCurrentIfNeeded(RepeatEdgeStep step, + boolean forward) { + step.decreaseTimes(); + if (forward) { + // Re-init sources + if (step.remainTimes() > 0) { + this.sources = this.newVertices; + } else { + this.sources = this.sourcesAll; + this.sourceFinishOneStep = true; + } } else { - this.sources = this.sourcesAll; - this.sourceFinishOneStep = true; + // Re-init targets + if (step.remainTimes() > 0) { + this.targets = this.newVertices; + } else { + this.targets = this.targetsAll; + this.targetFinishOneStep = true; + } } - this.stepCount++; } - /** - * Search backward from target - */ - public void backward() { - ConcurrentMultiValuedMap newVertices = - new ConcurrentMultiValuedMap<>(); + public RepeatEdgeStep forwardStep() { RepeatEdgeStep currentStep = null; - // Find next step to backward - for (int i = this.steps.size() - 1; i >= 0; i--) { + for (int i = 0; i < this.steps.size(); i++) { RepeatEdgeStep step = this.steps.get(i); if (step.remainTimes() > 0) { currentStep = step; @@ -298,86 +341,67 @@ public void backward() { break; } } - if (currentStep == null) { - return; - } + return currentStep; + } - // Re-init target all if last forward finished one super step and - // not last super step - if (this.targetFinishOneStep && !this.lastSuperStep()) { - this.targetsAll = new ConcurrentMultiValuedMap<>(); - this.targetFinishOneStep = false; + public RepeatEdgeStep backwardStep() { + RepeatEdgeStep currentStep = null; + // Find next step to backward + for (int i = this.steps.size() - 1; i >= 0; i--) { + RepeatEdgeStep step = this.steps.get(i); + if (step.remainTimes() > 0) { + currentStep = step; + this.targetIndex = i; + break; + } } + return currentStep; + } - currentStep.swithDirection(); - // Traversal vertices of previous level - RepeatEdgeStep finalCurrentStep = currentStep; - traverseIds(this.targets.keySet().iterator(), vid -> { - if (this.reachLimit()) { - return; - } - Iterator edges = edgesOfVertex(vid, finalCurrentStep); + public boolean lastSuperStep() { + return this.targetIndex == this.sourceIndex || + this.targetIndex == this.sourceIndex + 1; + } - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); + public void clearNewVertices() { + this.newVertices = this.newMultiValueMap(); + } - for (Node n : this.targets.get(vid)) { - // If have loop, skip target - if (n.contains(target)) { - continue; - } + public void addNodeToNewVertices(Id id, Node node) { + this.addNode(this.newVertices, id, node); + } - // If cross point exists, path found, concat them - if (this.lastStep() && - this.sources.containsKey(target)) { - for (Node node : this.sources.get(target)) { - List path = n.joinPath(node); - if (!path.isEmpty()) { - Path newPath = new Path(target, path); - newPath.reverse(); - this.paths.add(newPath); - if (this.reachLimit()) { - return; - } - } - } - } + public abstract Map> newMultiValueMap(); - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); - } - } - }); + public abstract Set newPathSet(); - currentStep.swithDirection(); + public abstract void addNode(Map> vertices, + Id id, Node node); - for (Map.Entry> entry : newVertices.entrySet()) { - this.targetsAll.addAll(entry.getKey(), entry.getValue()); - } + public abstract void addNewVerticesToAll(Map> targets); - currentStep.decreaseTimes(); - // Re-init targets - if (currentStep.remainTimes() > 0) { - this.targets = newVertices; - } else { - this.targets = this.targetsAll; - this.targetFinishOneStep = true; - } - this.stepCount++; - } - - @Override public Set paths() { return this.paths; } - @Override public int pathCount() { return this.paths.size(); } - protected int accessedNodes() { + protected boolean finish() { + return this.stepCount >= this.totalSteps || this.reachLimit(); + } + + protected boolean reachLimit() { + checkCapacity(this.capacity, this.accessedNodes(), + "template paths"); + if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { + return false; + } + return true; + } + + private int accessedNodes() { int size = 0; for (List value : this.sourcesAll.values()) { size += value.size(); @@ -389,228 +413,89 @@ protected int accessedNodes() { } } - private class SingleTraverser extends Traverser { - - private MultivaluedMap sources = newMultivalueMap(); - private MultivaluedMap targets = newMultivalueMap(); - private MultivaluedMap sourcesAll = newMultivalueMap(); - private MultivaluedMap targetsAll = newMultivalueMap(); - - private Set paths; + private class ConcurrentTraverser extends Traverser { - public SingleTraverser(Collection sources, Collection targets, - List steps, boolean withRing, - long capacity, long limit) { - super(steps, capacity, limit, withRing); - for (Id id : sources) { - this.sources.add(id, new Node(id)); - } - for (Id id : targets) { - this.targets.add(id, new Node(id)); - } - this.paths = new PathSet(); + public ConcurrentTraverser(Collection sources, + Collection targets, + List steps, boolean withRing, + long capacity, long limit) { + super(sources, targets, steps, withRing, capacity, limit); } - /** - * Search forward from sources - */ - public void forward() { - MultivaluedMap newVertices = newMultivalueMap(); - RepeatEdgeStep currentStep = null; - // Find next step to forward - for (int i = 0; i < this.steps.size(); i++) { - RepeatEdgeStep step = this.steps.get(i); - if (step.remainTimes() > 0) { - currentStep = step; - this.sourceIndex = i; - break; - } - } - if (currentStep == null) { - return; - } - - // Re-init source all if last forward finished one super step and - // not last super step - if (this.sourceFinishOneStep && !this.lastSuperStep()) { - this.sourcesAll = newMultivalueMap(); - this.sourceFinishOneStep = false; - } - - Iterator edges; - // Traversal vertices of previous level - for (Map.Entry> entry : this.sources.entrySet()) { - Id vid = entry.getKey(); - edges = edgesOfVertex(vid, currentStep); - - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - for (Node n : entry.getValue()) { - // If have loop, skip target - if (!this.withRing && n.contains(target)) { - continue; - } - - // If cross point exists, path found, concat them - if (this.lastSuperStep() && - this.targetsAll.containsKey(target)) { - for (Node node : this.targetsAll.get(target)) { - List path = joinPath(n, node, withRing); - if (!path.isEmpty()) { - this.paths.add(new Path(target, path)); - if (this.reachLimit()) { - return; - } - } - } - } - - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); - } - } - } - - for (Map.Entry> entry : newVertices.entrySet()) { - this.sourcesAll.addAll(entry.getKey(), entry.getValue()); - } - - currentStep.decreaseTimes(); - // Re-init sources - if (currentStep.remainTimes() > 0) { - this.sources = newVertices; - } else { - this.sources = this.sourcesAll; - this.sourceFinishOneStep = true; - } - this.stepCount++; + @Override + public Map> newMultiValueMap() { + return new ConcurrentMultiValuedMap<>(); } - /** - * Search backward from target - */ - public void backward() { - MultivaluedMap newVertices = newMultivalueMap(); - RepeatEdgeStep currentStep = null; - - // Find next step to backward - for (int i = this.steps.size() - 1; i >= 0; i--) { - RepeatEdgeStep step = this.steps.get(i); - if (step.remainTimes() > 0) { - currentStep = step; - this.targetIndex = i; - break; - } - } - if (currentStep == null) { - return; - } - - // Re-init target all if last forward finished one super step and - // not last super step - if (this.targetFinishOneStep && !this.lastSuperStep()) { - this.targetsAll = newMultivalueMap(); - this.targetFinishOneStep = false; - } + @Override + public void traverseOneLayer( + Map> vertices, RepeatEdgeStep step, + BiConsumer biConsumer) { + traverseIds(this.sources.keySet().iterator(), (id) -> { + biConsumer.accept(id, step); + }); + } - currentStep.swithDirection(); - Iterator edges; - // Traversal vertices of previous level - for (Map.Entry> entry : this.targets.entrySet()) { - Id vid = entry.getKey(); - edges = edgesOfVertex(vid, currentStep); - - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - for (Node n : entry.getValue()) { - // If have loop, skip target - if (!this.withRing && n.contains(target)) { - continue; - } + @Override + public Set newPathSet() { + return ConcurrentHashMap.newKeySet(); + } - // If cross point exists, path found, concat them - if (this.lastSuperStep() && - this.sourcesAll.containsKey(target)) { - for (Node node : this.sourcesAll.get(target)) { - List path = joinPath(n, node, withRing); - if (!path.isEmpty()) { - Path newPath = new Path(target, path); - newPath.reverse(); - this.paths.add(newPath); - if (this.reachLimit()) { - return; - } - } - } - } + @Override + public void addNode(Map> vertices, Id id, Node node) { + ((ConcurrentMultiValuedMap) vertices).add(id, node); + } - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); - } - } + @Override + public void addNewVerticesToAll(Map> targets) { + ConcurrentMultiValuedMap vertices = + (ConcurrentMultiValuedMap) targets; + for (Map.Entry> entry : this.newVertices.entrySet()) { + vertices.addAll(entry.getKey(), entry.getValue()); } + } + } - currentStep.swithDirection(); - - for (Map.Entry> entry : newVertices.entrySet()) { - this.targetsAll.addAll(entry.getKey(), entry.getValue()); - } + private class SingleTraverser extends Traverser { - currentStep.decreaseTimes(); - // Re-init targets - if (currentStep.remainTimes() > 0) { - this.targets = newVertices; - } else { - this.targets = this.targetsAll; - this.targetFinishOneStep = true; - } - this.stepCount++; + public SingleTraverser(Collection sources, Collection targets, + List steps, boolean withRing, + long capacity, long limit) { + super(sources, targets, steps, withRing, capacity, limit); } @Override - public Set paths() { - return this.paths; + public Map> newMultiValueMap() { + return newMultivalueMap(); } @Override - public int pathCount() { - return this.paths.size(); + public Set newPathSet() { + return new PathSet(); } - protected int accessedNodes() { - int size = 0; - for (List value : this.sourcesAll.values()) { - size += value.size(); - } - for (List value : this.targetsAll.values()) { - size += value.size(); + @Override + public void traverseOneLayer( + Map> vertices, RepeatEdgeStep step, + BiConsumer biConsumer) { + for (Id id : vertices.keySet()) { + biConsumer.accept(id, step); } - return size; } - } - - public static List joinPath(Node pre, Node back, boolean ring) { - // Get self path - List path = pre.path(); - // Get reversed other path - List backPath = back.path(); - Collections.reverse(backPath); + @Override + public void addNode(Map> vertices, Id id, Node node) { + ((MultivaluedMap) vertices).add(id, node); + } - if (!ring) { - // Avoid loop in path - if (CollectionUtils.containsAny(path, backPath)) { - return ImmutableList.of(); + @Override + public void addNewVerticesToAll(Map> targets) { + MultivaluedMap vertices = + (MultivaluedMap) targets; + for (Map.Entry> entry : this.newVertices.entrySet()) { + vertices.addAll(entry.getKey(), entry.getValue()); } } - - // Append other path behind self path - path.addAll(backPath); - return path; } public static class RepeatEdgeStep extends EdgeStep { From 21ea44cac2593462828060d846c5192bc76a78dd Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Mon, 12 Oct 2020 21:10:10 +0800 Subject: [PATCH 16/20] improve Change-Id: I7d6912f33836ac9a36fd0dda82b446780675a390 --- .../baidu/hugegraph/core/GraphManager.java | 2 +- .../hugegraph/serializer/JsonSerializer.java | 3 +- .../baidu/hugegraph/config/CoreOptions.java | 1 - .../CustomizedKneighborTraverser.java | 55 +++----------- .../algorithm/CustomizedKoutTraverser.java | 62 ++++----------- .../FusiformSimilarityTraverser.java | 4 +- .../traversal/algorithm/HugeTraverser.java | 33 +++++--- .../algorithm/TemplatePathsTraverser.java | 76 ++++++++++--------- .../traversal/algorithm/TpTraverser.java | 11 +++ 9 files changed, 100 insertions(+), 147 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java index c67192cda6..d50df8c056 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java @@ -165,7 +165,7 @@ private HugeAuthenticator authenticator() { @SuppressWarnings("unused") private void installLicense(HugeConfig config, String md5) { -// LicenseVerifier.instance().install(config, this, md5); + LicenseVerifier.instance().install(config, this, md5); } private void closeTx(final Set graphSourceNamesToCloseTxOn, diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java index 2b09b9f915..e9dc3f2713 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/serializer/JsonSerializer.java @@ -294,8 +294,7 @@ public String writeNodesWithPath(String name, Set nodes, Collection paths, Iterator iterator, boolean countOnly) { - List> pathList; - pathList = new ArrayList<>(); + List> pathList = new ArrayList<>(); for (HugeTraverser.Path path : paths) { pathList.add(path.toMap(false)); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java index d8d02b48f6..38d6321ded 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java @@ -519,5 +519,4 @@ public static synchronized CoreOptions instance() { rangeInt(0, 65535), 10 ); - } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java index 6c810b92c7..f4c9e1e4c0 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java @@ -21,8 +21,6 @@ import java.util.Iterator; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicLong; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; @@ -42,50 +40,16 @@ public Set customizedKneighbor(Id source, EdgeStep step, checkPositive(maxDepth, "k-neighbor max_depth"); checkLimit(limit); - if (maxDepth >= this.concurrentDepth() && - step.direction == Directions.BOTH) { - return this.customizedKneighborConcurrent(source, step, - maxDepth, limit); - } else { - return this.customizedKneighborSingle(source, step, - maxDepth, limit); - } - } - - public Set customizedKneighborConcurrent(Id source, EdgeStep step, - int maxDepth, long limit) { - Set latest = ConcurrentHashMap.newKeySet(); - Set all = ConcurrentHashMap.newKeySet(); - - Node sourceV = new KNode(source, null); - - latest.add(sourceV); - all.add(sourceV); - - while (maxDepth-- > 0) { - long remaining = limit == NO_LIMIT ? NO_LIMIT : limit - all.size(); - AtomicLong remain = new AtomicLong(remaining); - latest = this.adjacentVertices(latest, step, all, remain); - int size = all.size() + latest.size(); - if (limit != NO_LIMIT && size >= limit) { - int subLength = (int) limit - all.size(); - Iterator iterator = latest.iterator(); - for (int i = 0; i < subLength && iterator.hasNext(); i++) { - all.add(iterator.next()); - } - break; - } else { - all.addAll(latest); - } - } - - return all; + boolean single = maxDepth < this.concurrentDepth() || + step.direction != Directions.BOTH; + return this.customizedKneighbor(source, step, maxDepth, + limit, single); } - public Set customizedKneighborSingle(Id source, EdgeStep step, - int maxDepth, long limit) { - Set latest = newSet(); - Set all = newSet(); + public Set customizedKneighbor(Id source, EdgeStep step, int maxDepth, + long limit, boolean single) { + Set latest = newSet(single); + Set all = newSet(single); Node sourceV = new KNode(source, null); @@ -94,7 +58,8 @@ public Set customizedKneighborSingle(Id source, EdgeStep step, while (maxDepth-- > 0) { long remaining = limit == NO_LIMIT ? NO_LIMIT : limit - all.size(); - latest = this.adjacentVertices(latest, step, all, remaining); + latest = this.adjacentVertices(latest, step, all, + remaining, single); int size = all.size() + latest.size(); if (limit != NO_LIMIT && size >= limit) { int subLength = (int) limit - all.size(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java index 4e0c521eba..17a8bce2b7 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java @@ -20,8 +20,6 @@ package com.baidu.hugegraph.traversal.algorithm; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicLong; import com.baidu.hugegraph.HugeException; import com.baidu.hugegraph.HugeGraph; @@ -46,14 +44,10 @@ public Set customizedKout(Id source, EdgeStep step, int maxDepth, checkLimit(limit); Set results; - if (maxDepth >= this.concurrentDepth() && - step.direction == Directions.BOTH) { - results = this.customizedKoutConcurrent(source, step, maxDepth, - nearest, capacity); - } else { - results = this.customizedKoutSingle(source, step, maxDepth, - nearest, capacity); - } + boolean single = maxDepth < this.concurrentDepth() || + step.direction != Directions.BOTH; + results = this.customizedKout(source, step, maxDepth, nearest, + capacity, single); if (limit != NO_LIMIT && results.size() > limit) { results = CollectionUtil.subSet(results, 0, (int) limit); @@ -62,43 +56,11 @@ public Set customizedKout(Id source, EdgeStep step, int maxDepth, return results; } - public Set customizedKoutConcurrent(Id source, EdgeStep step, - int maxDepth, boolean nearest, - long capacity) { - Set latest = ConcurrentHashMap.newKeySet(); - Set all = ConcurrentHashMap.newKeySet(); - - Node sourceV = new KNode(source, null); - - latest.add(sourceV); - all.add(sourceV); - - int depth = maxDepth; - long remaining = capacity == NO_LIMIT ? - NO_LIMIT : capacity - latest.size(); - while (depth-- > 0) { - AtomicLong remain = new AtomicLong(remaining); - if (nearest) { - latest = this.adjacentVertices(latest, step, all, remain); - all.addAll(latest); - } else { - latest = this.adjacentVertices(latest, step, null, remain); - } - if (capacity != NO_LIMIT) { - // Update 'remaining' value to record remaining capacity - remaining -= latest.size(); - reachCapacity(remaining, capacity, depth); - } - } - - return latest; - } - - public Set customizedKoutSingle(Id source, EdgeStep step, - int maxDepth, boolean nearest, - long capacity) { - Set latest = newSet(); - Set all = newSet(); + public Set customizedKout(Id source, EdgeStep step, int maxDepth, + boolean nearest, long capacity, + boolean single) { + Set latest = newSet(single); + Set all = newSet(single); Node sourceV = new KNode(source, null); @@ -110,10 +72,12 @@ public Set customizedKoutSingle(Id source, EdgeStep step, NO_LIMIT : capacity - latest.size(); while (depth-- > 0) { if (nearest) { - latest = this.adjacentVertices(latest, step, all, remaining); + latest = this.adjacentVertices(latest, step, all, + remaining, single); all.addAll(latest); } else { - latest = this.adjacentVertices(latest, step, null, remaining); + latest = this.adjacentVertices(latest, step, null, + remaining, single); } if (capacity != NO_LIMIT) { // Update 'remaining' value to record remaining capacity diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java index f601b22602..17a534ab0e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/FusiformSimilarityTraverser.java @@ -196,8 +196,8 @@ private Set fusiformSimilarityForVertex( private static void checkGroupArgs(String groupProperty, int minGroups) { if (groupProperty == null) { E.checkArgument(minGroups == 0, - "Can not set min group count when " + - "group property not set"); + "Can't set min group count when " + + "group property not set"); } else { E.checkArgument(!groupProperty.isEmpty(), "The group property can't be empty"); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java index a2083a6661..431d7f2604 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import javax.ws.rs.core.MultivaluedHashMap; import javax.ws.rs.core.MultivaluedMap; @@ -262,6 +263,15 @@ protected Iterator adjacentVertices(Id source, Directions dir, }); } + protected Set adjacentVertices(Id source, EdgeStep step) { + Set neighbors = new HashSet<>(); + Iterator edges = this.edgesOfVertex(source, step); + while (edges.hasNext()) { + neighbors.add(((HugeEdge) edges.next()).id().otherVertexId()); + } + return neighbors; + } + protected Set adjacentVertices(Set vertices, EdgeStep step, Set excluded, long remaining) { Set neighbors = newSet(); @@ -282,15 +292,6 @@ protected Set adjacentVertices(Set vertices, EdgeStep step, return neighbors; } - protected Set adjacentVertices(Id source, EdgeStep step) { - Set neighbors = new HashSet<>(); - Iterator edges = this.edgesOfVertex(source, step); - while (edges.hasNext()) { - neighbors.add(((HugeEdge) edges.next()).id().otherVertexId()); - } - return neighbors; - } - protected Iterator edgesOfVertex(Id source, Directions dir, Id label, long limit) { Id[] labels = {}; @@ -540,7 +541,15 @@ public static Iterator skipSuperNodeIfNeeded(Iterator edges, } protected static Set newSet() { - return new HashSet<>(); + return newSet(true); + } + + protected static Set newSet(boolean single) { + if (single) { + return new HashSet<>(); + } else { + return ConcurrentHashMap.newKeySet(); + } } protected static Map newMap() { @@ -551,9 +560,9 @@ protected static MultivaluedMap newMultivalueMap() { return new MultivaluedHashMap<>(); } - protected static List joinPath(Node pre, Node back, boolean ring) { + protected static List joinPath(Node prev, Node back, boolean ring) { // Get self path - List path = pre.path(); + List path = prev.path(); // Get reversed other path List backPath = back.path(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index b7183282ac..1ec1720351 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -155,7 +155,7 @@ public Traverser(Collection sources, Collection targets, } public void forward() { - RepeatEdgeStep currentStep = this.step(true); + RepeatEdgeStep currentStep = this.nextStep(true); if (currentStep == null) { return; } @@ -169,7 +169,7 @@ public void forward() { } public void backward() { - RepeatEdgeStep currentStep = this.step(false); + RepeatEdgeStep currentStep = this.nextStep(false); if (currentStep == null) { return; } @@ -184,7 +184,7 @@ public void backward() { this.afterTraverse(currentStep, false); } - public RepeatEdgeStep step(boolean forward) { + public RepeatEdgeStep nextStep(boolean forward) { return forward ? this.forwardStep() : this.backwardStep(); } @@ -202,7 +202,7 @@ public void afterTraverse(RepeatEdgeStep step, boolean forward) { Map> all = forward ? this.sourcesAll : this.targetsAll; this.addNewVerticesToAll(all); - this.reInitCurrentIfNeeded(step, forward); + this.reInitCurrentStepIfNeeded(step, forward); this.stepCount++; } @@ -214,17 +214,18 @@ private void backward(Id v, RepeatEdgeStep step) { this.traverseOne(v, step, false); } - private void traverseOne(Id v, RepeatEdgeStep step, boolean forward) { + private void traverseOne(Id source, RepeatEdgeStep step, + boolean forward) { if (this.reachLimit()) { return; } - Iterator edges = edgesOfVertex(v, step); + Iterator edges = edgesOfVertex(source, step); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); Id target = edge.id().otherVertexId(); - this.processOne(v, target, forward); + this.processOne(source, target, forward); } } @@ -236,20 +237,20 @@ private void processOne(Id source, Id target, boolean forward) { } } - private void processOneForForward(Id source, Id target) { - for (Node n : this.sources.get(source)) { + private void processOneForForward(Id sourceV, Id targetV) { + for (Node source : this.sources.get(sourceV)) { // If have loop, skip target - if (!this.withRing && n.contains(target)) { + if (!this.withRing && source.contains(targetV)) { continue; } // If cross point exists, path found, concat them if (this.lastSuperStep() && - this.targetsAll.containsKey(target)) { - for (Node node : this.targetsAll.get(target)) { - List path = joinPath(n, node, this.withRing); + this.targetsAll.containsKey(targetV)) { + for (Node target : this.targetsAll.get(targetV)) { + List path = joinPath(source, target, this.withRing); if (!path.isEmpty()) { - this.paths.add(new Path(target, path)); + this.paths.add(new Path(targetV, path)); if (this.reachLimit()) { return; } @@ -258,24 +259,24 @@ private void processOneForForward(Id source, Id target) { } // Add node to next start-nodes - this.addNodeToNewVertices(target, new Node(target, n)); + this.addNodeToNewVertices(targetV, new Node(targetV, source)); } } - private void processOneForBackward(Id source, Id target) { - for (Node n : this.targets.get(source)) { + private void processOneForBackward(Id sourceV, Id targetV) { + for (Node source : this.targets.get(sourceV)) { // If have loop, skip target - if (!this.withRing && n.contains(target)) { + if (!this.withRing && source.contains(targetV)) { continue; } // If cross point exists, path found, concat them if (this.lastSuperStep() && - this.sourcesAll.containsKey(target)) { - for (Node node : this.sourcesAll.get(target)) { - List path = joinPath(n, node, this.withRing); + this.sourcesAll.containsKey(targetV)) { + for (Node target : this.sourcesAll.get(targetV)) { + List path = joinPath(source, target, this.withRing); if (!path.isEmpty()) { - Path newPath = new Path(target, path); + Path newPath = new Path(targetV, path); newPath.reverse(); this.paths.add(newPath); if (this.reachLimit()) { @@ -286,21 +287,25 @@ private void processOneForBackward(Id source, Id target) { } // Add node to next start-nodes - this.addNodeToNewVertices(target, new Node(target, n)); + this.addNodeToNewVertices(targetV, new Node(targetV, source)); } } private void reInitAllIfNeeded(boolean forward) { if (forward) { - // Re-init source all if last forward finished one super step - // and not last super step + /* + * Re-init source all if last forward finished one super step + * and current step is not last super step + */ if (this.sourceFinishOneStep && !this.lastSuperStep()) { this.sourcesAll = this.newMultiValueMap(); this.sourceFinishOneStep = false; } } else { - // Re-init target all if last forward finished one super step - // and not last super step + /* + * Re-init target all if last forward finished one super step + * and current step is not last super step + */ if (this.targetFinishOneStep && !this.lastSuperStep()) { this.targetsAll = this.newMultiValueMap(); this.targetFinishOneStep = false; @@ -308,8 +313,8 @@ private void reInitAllIfNeeded(boolean forward) { } } - private void reInitCurrentIfNeeded(RepeatEdgeStep step, - boolean forward) { + private void reInitCurrentStepIfNeeded(RepeatEdgeStep step, + boolean forward) { step.decreaseTimes(); if (forward) { // Re-init sources @@ -417,7 +422,8 @@ private class ConcurrentTraverser extends Traverser { public ConcurrentTraverser(Collection sources, Collection targets, - List steps, boolean withRing, + List steps, + boolean withRing, long capacity, long limit) { super(sources, targets, steps, withRing, capacity, limit); } @@ -430,9 +436,9 @@ public Map> newMultiValueMap() { @Override public void traverseOneLayer( Map> vertices, RepeatEdgeStep step, - BiConsumer biConsumer) { - traverseIds(this.sources.keySet().iterator(), (id) -> { - biConsumer.accept(id, step); + BiConsumer consumer) { + traverseIds(this.sources.keySet().iterator(), id -> { + consumer.accept(id, step); }); } @@ -477,9 +483,9 @@ public Set newPathSet() { @Override public void traverseOneLayer( Map> vertices, RepeatEdgeStep step, - BiConsumer biConsumer) { + BiConsumer consumer) { for (Id id : vertices.keySet()) { - biConsumer.accept(id, step); + consumer.accept(id, step); } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index 7b630d1ba3..e644e3a781 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -83,6 +83,17 @@ public void close() { } } + protected Set adjacentVertices(Set latest, EdgeStep step, + Set all, long remaining, + boolean single) { + if (single) { + return this.adjacentVertices(latest, step, all, remaining); + } else { + AtomicLong remain = new AtomicLong(remaining); + return this.adjacentVertices(latest, step, all, remain); + } + } + protected Set adjacentVertices(Set vertices, EdgeStep step, Set excluded, AtomicLong remaining) { From 876b3e1584d768ec09572a41f6726b1f9b2c6f6f Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Tue, 13 Oct 2020 20:31:12 +0800 Subject: [PATCH 17/20] extract path traverser for template paths and collection paths Change-Id: I5b7441f562106fa3810d6d844c897add3010ef30 --- .../api/traversers/JaccardSimilarityAPI.java | 3 +- .../api/traversers/KneighborAPI.java | 7 +- .../hugegraph/api/traversers/KoutAPI.java | 12 +- .../api/traversers/SameNeighborsAPI.java | 4 +- .../algorithm/CollectionPathsTraverser.java | 594 +++++------------- .../traversal/algorithm/HugeTraverser.java | 144 +---- .../algorithm/JaccardSimilarTraverser.java | 28 + ...Traverser.java => KneighborTraverser.java} | 35 +- ...dKoutTraverser.java => KoutTraverser.java} | 61 +- .../algorithm/SameNeighborTraverser.java | 62 ++ .../algorithm/TemplatePathsTraverser.java | 280 +-------- .../traversal/algorithm/TpTraverser.java | 199 ++++++ .../strategy/ConcurrentTraverseStrategy.java | 74 +++ .../strategy/SingleTraverseStrategy.java | 75 +++ .../algorithm/strategy/TraverseStrategy.java | 47 ++ 15 files changed, 771 insertions(+), 854 deletions(-) rename hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/{CustomizedKneighborTraverser.java => KneighborTraverser.java} (71%) rename hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/{CustomizedKoutTraverser.java => KoutTraverser.java} (60%) create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SameNeighborTraverser.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java index 8de41fa3c3..c7ee87525f 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/JaccardSimilarityAPI.java @@ -46,7 +46,6 @@ import com.baidu.hugegraph.server.RestServer; import com.baidu.hugegraph.structure.HugeVertex; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; -import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.traversal.algorithm.JaccardSimilarTraverser; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; @@ -82,7 +81,7 @@ public String get(@Context GraphManager manager, Directions dir = Directions.convert(EdgeAPI.parseDirection(direction)); HugeGraph g = graph(manager, graph); - HugeTraverser traverser = new HugeTraverser(g); + JaccardSimilarTraverser traverser = new JaccardSimilarTraverser(g); double similarity = traverser.jaccardSimilarity(sourceId, targetId, dir, edgeLabel, degree); return JsonUtil.toJson(ImmutableMap.of("jaccard_similarity", diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java index 58cddfca31..77822e7c3b 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KneighborAPI.java @@ -51,7 +51,7 @@ import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; import com.baidu.hugegraph.structure.HugeVertex; -import com.baidu.hugegraph.traversal.algorithm.CustomizedKneighborTraverser; +import com.baidu.hugegraph.traversal.algorithm.KneighborTraverser; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.type.define.Directions; @@ -90,7 +90,7 @@ public String get(@Context GraphManager manager, HugeGraph g = graph(manager, graph); - HugeTraverser traverser = new HugeTraverser(g); + KneighborTraverser traverser = new KneighborTraverser(g); Set ids = traverser.kneighbor(source, dir, edgeLabel, depth, degree, limit); return manager.serializer(g).writeList("vertices", ids); @@ -124,8 +124,7 @@ public String post(@Context GraphManager manager, EdgeStep step = step(g, request.step); - CustomizedKneighborTraverser traverser = - new CustomizedKneighborTraverser(g); + KneighborTraverser traverser = new KneighborTraverser(g); Set results = traverser.customizedKneighbor( sourceId, step, request.maxDepth, request.limit); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java index dd341c87ad..eb7aa401f8 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/KoutAPI.java @@ -47,7 +47,7 @@ import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; import com.baidu.hugegraph.structure.HugeVertex; -import com.baidu.hugegraph.traversal.algorithm.CustomizedKoutTraverser; +import com.baidu.hugegraph.traversal.algorithm.KoutTraverser; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.type.define.Directions; @@ -56,7 +56,11 @@ import com.codahale.metrics.annotation.Timed; import com.fasterxml.jackson.annotation.JsonProperty; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.*; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.Node; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_CAPACITY; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_DEGREE; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_ELEMENTS_LIMIT; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_PATHS_LIMIT; @Path("graphs/{graph}/traversers/kout") @Singleton @@ -92,7 +96,7 @@ public String get(@Context GraphManager manager, HugeGraph g = graph(manager, graph); - HugeTraverser traverser = new HugeTraverser(g); + KoutTraverser traverser = new KoutTraverser(g); Set ids = traverser.kout(sourceId, dir, edgeLabel, depth, nearest, degree, capacity, limit); return manager.serializer(g).writeList("vertices", ids); @@ -128,7 +132,7 @@ public String post(@Context GraphManager manager, EdgeStep step = step(g, request.step); - CustomizedKoutTraverser traverser = new CustomizedKoutTraverser(g); + KoutTraverser traverser = new KoutTraverser(g); Set results = traverser.customizedKout( sourceId, step, request.maxDepth, request.nearest, request.capacity, diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/SameNeighborsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/SameNeighborsAPI.java index ca449c4564..0c6c3c1b83 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/SameNeighborsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/SameNeighborsAPI.java @@ -42,7 +42,7 @@ import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; -import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; +import com.baidu.hugegraph.traversal.algorithm.SameNeighborTraverser; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.Log; import com.codahale.metrics.annotation.Timed; @@ -75,7 +75,7 @@ public String get(@Context GraphManager manager, Directions dir = Directions.convert(EdgeAPI.parseDirection(direction)); HugeGraph g = graph(manager, graph); - HugeTraverser traverser = new HugeTraverser(g); + SameNeighborTraverser traverser = new SameNeighborTraverser(g); Set neighbors = traverser.sameNeighbors(sourceId, targetId, dir, edgeLabel, degree, limit); return manager.serializer(g).writeList("same_neighbors", neighbors); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java index 926dd63b55..2f5a0c23b4 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -21,25 +21,24 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import javax.ws.rs.core.MultivaluedMap; -import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Vertex; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; -import com.baidu.hugegraph.structure.HugeEdge; import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.strategy.ConcurrentTraverseStrategy; +import com.baidu.hugegraph.traversal.algorithm.strategy.SingleTraverseStrategy; +import com.baidu.hugegraph.traversal.algorithm.strategy.TraverseStrategy; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; +import com.google.common.collect.ImmutableList; public class CollectionPathsTraverser extends TpTraverser { @@ -49,7 +48,6 @@ public CollectionPathsTraverser(HugeGraph graph) { super(graph, "collection-paths"); } - @SuppressWarnings("unchecked") public Collection paths(Iterator sources, Iterator targets, EdgeStep step, int depth, boolean nearest, @@ -75,509 +73,215 @@ public Collection paths(Iterator sources, "but got: %s", MAX_VERTICES, sourceList.size()); checkPositive(depth, "max depth"); - Traverser traverser = depth >= this.concurrentDepth() ? - this.concurrentTraverser(sourceList, targetList, - step, nearest, - capacity, limit) : - this.singleTraverser(sourceList, targetList, step, - nearest, capacity, limit); - - while (true) { - if (--depth < 0 || traverser.reachLimit()) { - break; - } - traverser.forward(); - - if (--depth < 0 || traverser.reachLimit()) { - break; + Traverser traverser; + + TraverseStrategy strategy; + if (depth >= this.concurrentDepth()) { + strategy = new ConcurrentTraverseStrategy(this.graph()); + traverser = new Traverser(sourceList, targetList, step, depth, + capacity, limit, strategy); + } else { + strategy = new SingleTraverseStrategy(this.graph()); + if (nearest) { + traverser = new SingleNearestTraverser(sourceList, targetList, + step, depth, capacity, + limit, strategy); + } else { + traverser = new Traverser(sourceList, targetList, step, + depth, capacity, limit, strategy); } - traverser.backward(); } - return traverser.paths(); - } - private Traverser singleTraverser(List sources, List targets, - EdgeStep step, boolean nearest, - long capacity, long limit) { - return nearest ? new SingleNearestTraverser(sources, targets, step, - capacity, limit) : - new SingleAllTraverser(sources, targets, step, - capacity, limit); - } + do { + // Forward + traverser.forward(); + if (traverser.finish()) { + return traverser.paths(); + } - private Traverser concurrentTraverser(List sources, List targets, - EdgeStep step, boolean nearest, - long capacity, long limit) { - return new ConcurrentTraverser(sources, targets, step, capacity, limit); + // Backward + traverser.backward(); + if (traverser.finish()) { + return traverser.paths(); + } + } while (true); } - private class Traverser { + private class Traverser extends PathTraverser { protected final EdgeStep step; - protected final long capacity; - protected final long limit; - public Traverser(EdgeStep step, long capacity, long limit) { + public Traverser(Collection sources, Collection targets, + EdgeStep step, int depth, long capacity, + long limit, TraverseStrategy strategy) { + super(sources, targets, capacity, limit, strategy); this.step = step; - this.capacity = capacity; - this.limit = limit; - } - - public void forward() { - } - - public void backward() { + this.totalSteps = depth; } - public Set paths() { - return new PathSet(); - } - - public int pathCount() { - return 0; - } - - protected int accessedNodes() { - return 0; - } - - protected boolean reachLimit() { - checkCapacity(this.capacity, this.accessedNodes(), - "collection paths"); - if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { - return false; - } - return true; - } - } - - private class ConcurrentTraverser extends Traverser { - - private ConcurrentMultiValuedMap sources = - new ConcurrentMultiValuedMap<>(); - private ConcurrentMultiValuedMap targets = - new ConcurrentMultiValuedMap<>(); - private ConcurrentMultiValuedMap sourcesAll = - new ConcurrentMultiValuedMap<>(); - private ConcurrentMultiValuedMap targetsAll = - new ConcurrentMultiValuedMap<>(); - - private Set paths; - - public ConcurrentTraverser(Collection sources, - Collection targets, EdgeStep step, - long capacity, long limit) { - super(step, capacity, limit); - for (Id id : sources) { - this.sources.add(id, new Node(id)); - } - for (Id id : targets) { - this.targets.add(id, new Node(id)); - } - this.sourcesAll.putAll(this.sources); - this.targetsAll.putAll(this.targets); - - this.paths = ConcurrentHashMap.newKeySet(); + @Override + public EdgeStep nextStep(boolean forward) { + return this.step; } - /** - * Search forward from sources - */ - public void forward() { - ConcurrentMultiValuedMap newVertices = - new ConcurrentMultiValuedMap<>(); - // Traversal vertices of previous level - traverseIds(this.sources.keySet().iterator(), vid -> { - if (this.reachLimit()) { - return; + @Override + protected void processOneForForward(Id sourceV, Id targetV) { + for (Node source : this.sources.get(sourceV)) { + // If have loop, skip target + if (source.contains(targetV)) { + continue; } - Iterator edges = edgesOfVertex(vid, this.step); - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - for (Node n : this.sources.get(vid)) { - // If have loop, skip target - if (n.contains(target)) { - continue; - } - // If cross point exists, path found, concat them - if (this.targetsAll.containsKey(target)) { - for (Node node : this.targetsAll.get(target)) { - List path = n.joinPath(node); - if (!path.isEmpty()) { - this.paths.add(new Path(target, path)); - if (this.reachLimit()) { - return; - } - } + // If cross point exists, path found, concat them + if (this.targetsAll.containsKey(targetV)) { + for (Node target : this.targetsAll.get(targetV)) { + List path = source.joinPath(target); + if (!path.isEmpty()) { + this.paths.add(new Path(targetV, path)); + if (this.reachLimit()) { + return; } } - - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); } } - }); - // Re-init sources - this.sources = newVertices; - // Record all passed vertices - for (Map.Entry> entry : newVertices.entrySet()) { - this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + // Add node to next start-nodes + this.addNodeToNewVertices(targetV, new Node(targetV, source)); } } - /** - * Search backward from target - */ - public void backward() { - ConcurrentMultiValuedMap newVertices = - new ConcurrentMultiValuedMap<>(); - this.step.swithDirection(); - // Traversal vertices of previous level - traverseIds(this.targets.keySet().iterator(), vid -> { - if (this.reachLimit()) { - return; + @Override + protected void processOneForBackward(Id sourceV, Id targetV) { + for (Node source : this.targets.get(sourceV)) { + // If have loop, skip target + if (source.contains(targetV)) { + continue; } - Iterator edges = edgesOfVertex(vid, this.step); - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - for (Node n : this.targets.get(vid)) { - // If have loop, skip target - if (n.contains(target)) { - continue; - } - - // If cross point exists, path found, concat them - if (this.sourcesAll.containsKey(target)) { - for (Node node : this.sourcesAll.get(target)) { - List path = n.joinPath(node); - if (!path.isEmpty()) { - Path newPath = new Path(target, path); - newPath.reverse(); - this.paths.add(newPath); - if (this.reachLimit()) { - return; - } - } + // If cross point exists, path found, concat them + if (this.sourcesAll.containsKey(targetV)) { + for (Node target : this.sourcesAll.get(targetV)) { + List path = source.joinPath(target); + if (!path.isEmpty()) { + Path newPath = new Path(targetV, path); + newPath.reverse(); + this.paths.add(newPath); + if (this.reachLimit()) { + return; } } - - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); } } - }); - this.step.swithDirection(); - - // Re-init targets - this.targets = newVertices; - // Record all passed vertices - for (Map.Entry> entry : newVertices.entrySet()) { - this.targetsAll.addAll(entry.getKey(), entry.getValue()); - } - } - @Override - public Set paths() { - return this.paths; + // Add node to next start-nodes + this.addNodeToNewVertices(targetV, new Node(targetV, source)); + } } @Override - public int pathCount() { - return this.paths.size(); - } - - protected int accessedNodes() { - return this.sourcesAll.size() + this.targetsAll.size(); + protected void reInitCurrentStepIfNeeded(EdgeStep step, + boolean forward) { + if (forward) { + // Re-init sources + this.sources = this.newVertices; + // Record all passed vertices + this.addNewVerticesToAll(this.sourcesAll); + } else { + // Re-init targets + this.targets = this.newVertices; + // Record all passed vertices + this.addNewVerticesToAll(this.targetsAll); + } } } - private class SingleAllTraverser extends Traverser { - - private MultivaluedMap sources = newMultivalueMap(); - private MultivaluedMap targets = newMultivalueMap(); - private MultivaluedMap sourcesAll = newMultivalueMap(); - private MultivaluedMap targetsAll = newMultivalueMap(); - - private PathSet paths; + private class SingleNearestTraverser extends Traverser { - public SingleAllTraverser(Collection sources, - Collection targets, - EdgeStep step, long capacity, long limit) { - super(step, capacity, limit); - for (Id id : sources) { - this.sources.add(id, new Node(id)); - } - for (Id id : targets) { - this.targets.add(id, new Node(id)); - } - this.sourcesAll.putAll(this.sources); - this.targetsAll.putAll(this.targets); - this.paths = new PathSet(); + public SingleNearestTraverser(Collection sources, + Collection targets, EdgeStep step, + int depth, long capacity, long limit, + TraverseStrategy strategy) { + super(sources, targets, step, depth, capacity, limit, strategy); } - /** - * Search forward from sources - */ - public void forward() { - MultivaluedMap newVertices = newMultivalueMap(); - Iterator edges; - // Traversal vertices of previous level - for (Map.Entry> entry : this.sources.entrySet()) { - Id vid = entry.getKey(); - edges = edgesOfVertex(vid, this.step); - - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - for (Node n : entry.getValue()) { - // If have loop, skip target - if (n.contains(target)) { - continue; - } - - // If cross point exists, path found, concat them - if (this.targetsAll.containsKey(target)) { - for (Node node : this.targetsAll.get(target)) { - List path = n.joinPath(node); - if (!path.isEmpty()) { - this.paths.add(new Path(target, path)); - if (this.reachLimit()) { - return; - } - } - } - } - - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); - } - } - } - - // Re-init targets - this.sources = newVertices; - // Record all passed vertices - for (Map.Entry> entry : newVertices.entrySet()) { - this.sourcesAll.addAll(entry.getKey(), entry.getValue()); + @Override + protected void processOneForForward(Id sourceV, Id targetV) { + Node source = this.sources.get(sourceV).get(0); + // If have loop, skip target + if (source.contains(targetV)) { + return; } - } - - /** - * Search backward from target - */ - public void backward() { - PathSet paths = new PathSet(); - MultivaluedMap newVertices = newMultivalueMap(); - this.step.swithDirection(); - Iterator edges; - // Traversal vertices of previous level - for (Map.Entry> entry : this.targets.entrySet()) { - Id vid = entry.getKey(); - edges = edgesOfVertex(vid, this.step); - - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - for (Node n : entry.getValue()) { - // If have loop, skip target - if (n.contains(target)) { - continue; - } - - // If cross point exists, path found, concat them - if (this.sourcesAll.containsKey(target)) { - for (Node node : this.sourcesAll.get(target)) { - List path = n.joinPath(node); - if (!path.isEmpty()) { - Path newPath = new Path(target, path); - newPath.reverse(); - this.paths.add(newPath); - if (this.reachLimit()) { - return; - } - } - } - } - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); + // If cross point exists, path found, concat them + if (this.targetsAll.containsKey(targetV)) { + Node node = this.targetsAll.get(targetV).get(0); + List path = source.joinPath(node); + if (!path.isEmpty()) { + this.paths.add(new Path(targetV, path)); + if (this.reachLimit()) { + return; } } } - this.step.swithDirection(); - - // Re-init targets - this.targets = newVertices; - // Record all passed vertices - for (Map.Entry> entry : newVertices.entrySet()) { - this.targetsAll.addAll(entry.getKey(), entry.getValue()); - } - } - @Override - public Set paths() { - return this.paths; + // Add node to next start-nodes + this.addNodeToNewVertices(targetV, new Node(targetV, source)); } @Override - public int pathCount() { - return this.paths.size(); - } - - protected int accessedNodes() { - return this.sourcesAll.size() + this.targetsAll.size(); - } - } - - - private class SingleNearestTraverser extends Traverser { - - private Map sources = new HashMap<>(); - private Map targets = new HashMap<>(); - private Map sourcesAll = new HashMap<>(); - private Map targetsAll = new HashMap<>(); - - private PathSet paths; - - public SingleNearestTraverser(Collection sources, - Collection targets, - EdgeStep step, long capacity, - long limit) { - super(step, capacity, limit); - for (Id id : sources) { - this.sources.put(id, new KNode(id, null)); - } - for (Id id : targets) { - this.targets.put(id, new KNode(id, null)); + protected void processOneForBackward(Id sourceV, Id targetV) { + Node sourcee = this.targets.get(sourceV).get(0); + // If have loop, skip target + if (sourcee.contains(targetV)) { + return; } - this.sourcesAll.putAll(this.sources); - this.targetsAll.putAll(this.targets); - this.paths = new PathSet(); - } - /** - * Search forward from sources - */ - public void forward() { - LOG.info("Forward with sources size {} and sources all size {}", - this.sources.size(), this.sourcesAll.size()); - Map newVertices = new HashMap<>(); - Iterator edges; - // Traversal vertices of previous level - for (Map.Entry entry : this.sources.entrySet()) { - Id vid = entry.getKey(); - edges = edgesOfVertex(vid, this.step); - - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - Node n = entry.getValue(); - // If have loop, skip target - if (n.contains(target)) { - continue; + // If cross point exists, path found, concat them + if (this.sourcesAll.containsKey(targetV)) { + Node node = this.sourcesAll.get(targetV).get(0); + List path = sourcee.joinPath(node); + if (!path.isEmpty()) { + Path newPath = new Path(targetV, path); + newPath.reverse(); + this.paths.add(newPath); + if (this.reachLimit()) { + return; } - - // If cross point exists, path found, concat them - if (this.targetsAll.containsKey(target)) { - Node node = this.targetsAll.get(target); - List path = n.joinPath(node); - if (!path.isEmpty()) { - this.paths.add(new Path(target, path)); - if (this.reachLimit()) { - return; - } - } - } - - // Add node to next start-nodes - newVertices.putIfAbsent(target, - new KNode(target, (KNode) n)); } } - // Re-init targets - this.sources = newVertices; - // Record all passed vertices - for (Map.Entry entry : newVertices.entrySet()) { - this.sourcesAll.putIfAbsent(entry.getKey(), entry.getValue()); - } - LOG.info("Done forward with sources size {} and sources all size {}", - this.sources.size(), this.sourcesAll.size()); + // Add node to next start-nodes + this.addNodeToNewVertices(targetV, new Node(targetV, sourcee)); } - /** - * Search backward from target - */ - public void backward() { - LOG.info("Backward with targets size {} and targets all size {}", - this.targets.size(), this.targetsAll.size()); - Map newVertices = new HashMap<>(); - this.step.swithDirection(); - Iterator edges; - // Traversal vertices of previous level - for (Map.Entry entry : this.targets.entrySet()) { - Id vid = entry.getKey(); - edges = edgesOfVertex(vid, this.step); - - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - Node n = entry.getValue(); - // If have loop, skip target - if (n.contains(target)) { - continue; - } - - // If cross point exists, path found, concat them - if (this.sourcesAll.containsKey(target)) { - Node node = this.sourcesAll.get(target); - List path = n.joinPath(node); - if (!path.isEmpty()) { - Path newPath = new Path(target, path); - newPath.reverse(); - this.paths.add(newPath); - if (this.reachLimit()) { - return; - } - } - } - - // Add node to next start-nodes - newVertices.putIfAbsent(target, - new KNode(target, (KNode) n)); - } - } - this.step.swithDirection(); - - // Re-init targets - this.targets = newVertices; - // Record all passed vertices - for (Map.Entry entry : newVertices.entrySet()) { - this.targetsAll.putIfAbsent(entry.getKey(), entry.getValue()); + @Override + protected void reInitCurrentStepIfNeeded(EdgeStep step, + boolean forward) { + if (forward) { + // Re-init targets + this.sources = this.newVertices; + // Record all passed vertices + this.addNewVerticesToAll(this.sourcesAll); + } else { + // Re-init targets + this.targets = this.newVertices; + // Record all passed vertices + this.addNewVerticesToAll(this.targetsAll); } - LOG.info("Done backward with sources size {} and sources all size {}", - this.targets.size(), this.targetsAll.size()); } @Override - public Set paths() { - return this.paths; + public void addNodeToNewVertices(Id id, Node node) { + this.newVertices.putIfAbsent(id, ImmutableList.of(node)); } @Override - public int pathCount() { - return this.paths.size(); + public void addNewVerticesToAll(Map> targets) { + MultivaluedMap vertices = + (MultivaluedMap) targets; + for (Map.Entry> entry : this.newVertices.entrySet()) { + vertices.putIfAbsent(entry.getKey(), entry.getValue()); + } } protected int accessedNodes() { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java index 431d7f2604..3ef436eebe 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java @@ -35,7 +35,6 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.tinkerpop.gremlin.structure.Edge; -import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; import com.baidu.hugegraph.HugeException; import com.baidu.hugegraph.HugeGraph; @@ -91,146 +90,9 @@ public HugeGraph graph() { return this.graph; } - public Set kout(Id sourceV, Directions dir, String label, - int depth, boolean nearest, - long degree, long capacity, long limit) { - E.checkNotNull(sourceV, "source vertex id"); - this.checkVertexExist(sourceV, "source vertex"); - E.checkNotNull(dir, "direction"); - checkPositive(depth, "k-out max_depth"); - checkDegree(degree); - checkCapacity(capacity); - checkLimit(limit); - if (capacity != NO_LIMIT) { - // Capacity must > limit because sourceV is counted in capacity - E.checkArgument(capacity >= limit && limit != NO_LIMIT, - "Capacity can't be less than limit, " + - "but got capacity '%s' and limit '%s'", - capacity, limit); - } - - Id labelId = this.getEdgeLabelId(label); - - Set latest = newSet(); - latest.add(sourceV); - - Set all = newSet(); - all.add(sourceV); - - long remaining = capacity == NO_LIMIT ? - NO_LIMIT : capacity - latest.size(); - while (depth-- > 0) { - // Just get limit nodes in last layer if limit < remaining capacity - if (depth == 0 && limit != NO_LIMIT && - (limit < remaining || remaining == NO_LIMIT)) { - remaining = limit; - } - if (nearest) { - latest = this.adjacentVertices(latest, dir, labelId, all, - degree, remaining); - all.addAll(latest); - } else { - latest = this.adjacentVertices(latest, dir, labelId, null, - degree, remaining); - } - if (capacity != NO_LIMIT) { - // Update 'remaining' value to record remaining capacity - remaining -= latest.size(); - - if (remaining <= 0 && depth > 0) { - throw new HugeException( - "Reach capacity '%s' while remaining depth '%s'", - capacity, depth); - } - } - } - - return latest; - } - - public Set kneighbor(Id sourceV, Directions dir, - String label, int depth, - long degree, long limit) { - E.checkNotNull(sourceV, "source vertex id"); - this.checkVertexExist(sourceV, "source vertex"); - E.checkNotNull(dir, "direction"); - checkPositive(depth, "k-neighbor max_depth"); - checkDegree(degree); - checkLimit(limit); - - Id labelId = this.getEdgeLabelId(label); - - Set latest = newSet(); - latest.add(sourceV); - - Set all = newSet(); - all.add(sourceV); - - while (depth-- > 0) { - long remaining = limit == NO_LIMIT ? NO_LIMIT : limit - all.size(); - latest = this.adjacentVertices(latest, dir, labelId, all, - degree, remaining); - all.addAll(latest); - if (limit != NO_LIMIT && all.size() >= limit) { - break; - } - } - - return all; - } - - public Set sameNeighbors(Id vertex, Id other, Directions direction, - String label, long degree, long limit) { - E.checkNotNull(vertex, "vertex id"); - E.checkNotNull(other, "the other vertex id"); - this.checkVertexExist(vertex, "vertex"); - this.checkVertexExist(other, "other vertex"); - E.checkNotNull(direction, "direction"); - checkDegree(degree); - checkLimit(limit); - - Id labelId = this.getEdgeLabelId(label); - - Set sourceNeighbors = IteratorUtils.set(this.adjacentVertices( - vertex, direction, labelId, degree)); - Set targetNeighbors = IteratorUtils.set(this.adjacentVertices( - other, direction, labelId, degree)); - Set sameNeighbors = (Set) CollectionUtil.intersect( - sourceNeighbors, targetNeighbors); - if (limit != NO_LIMIT) { - int end = Math.min(sameNeighbors.size(), (int) limit); - sameNeighbors = CollectionUtil.subSet(sameNeighbors, 0, end); - } - return sameNeighbors; - } - - public double jaccardSimilarity(Id vertex, Id other, Directions dir, - String label, long degree) { - E.checkNotNull(vertex, "vertex id"); - E.checkNotNull(other, "the other vertex id"); - this.checkVertexExist(vertex, "vertex"); - this.checkVertexExist(other, "other vertex"); - E.checkNotNull(dir, "direction"); - checkDegree(degree); - - Id labelId = this.getEdgeLabelId(label); - - Set sourceNeighbors = IteratorUtils.set(this.adjacentVertices( - vertex, dir, labelId, degree)); - Set targetNeighbors = IteratorUtils.set(this.adjacentVertices( - other, dir, labelId, degree)); - return jaccardSimilarity(sourceNeighbors, targetNeighbors); - } - - public double jaccardSimilarity(Set set1, Set set2) { - int interNum = CollectionUtil.intersect(set1, set2).size(); - int unionNum = CollectionUtil.union(set1, set2).size(); - return (double) interNum / unionNum; - } - - private Set adjacentVertices(Set vertices, Directions dir, - Id label, Set excluded, - long degree, long limit) { + protected Set adjacentVertices(Set vertices, Directions dir, + Id label, Set excluded, + long degree, long limit) { if (limit == 0) { return ImmutableSet.of(); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java index 59fbadfb46..321de5b064 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java @@ -26,10 +26,13 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; + import com.baidu.hugegraph.HugeException; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; import com.google.common.collect.ImmutableMap; @@ -39,6 +42,31 @@ public JaccardSimilarTraverser(HugeGraph graph) { super(graph, "jaccard-similar"); } + + public double jaccardSimilarity(Id vertex, Id other, Directions dir, + String label, long degree) { + E.checkNotNull(vertex, "vertex id"); + E.checkNotNull(other, "the other vertex id"); + this.checkVertexExist(vertex, "vertex"); + this.checkVertexExist(other, "other vertex"); + E.checkNotNull(dir, "direction"); + checkDegree(degree); + + Id labelId = this.getEdgeLabelId(label); + + Set sourceNeighbors = IteratorUtils.set(this.adjacentVertices( + vertex, dir, labelId, degree)); + Set targetNeighbors = IteratorUtils.set(this.adjacentVertices( + other, dir, labelId, degree)); + return jaccardSimilarity(sourceNeighbors, targetNeighbors); + } + + public double jaccardSimilarity(Set set1, Set set2) { + int interNum = CollectionUtil.intersect(set1, set2).size(); + int unionNum = CollectionUtil.union(set1, set2).size(); + return (double) interNum / unionNum; + } + public Map jaccardSimilars(Id source, EdgeStep step, int top, long capacity) { E.checkNotNull(source, "source vertex id"); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java similarity index 71% rename from hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java rename to hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java index f4c9e1e4c0..2210653661 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKneighborTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java @@ -27,12 +27,43 @@ import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; -public class CustomizedKneighborTraverser extends TpTraverser { +public class KneighborTraverser extends TpTraverser { - public CustomizedKneighborTraverser(HugeGraph graph) { + public KneighborTraverser(HugeGraph graph) { super(graph, "kneighbor"); } + public Set kneighbor(Id sourceV, Directions dir, + String label, int depth, + long degree, long limit) { + E.checkNotNull(sourceV, "source vertex id"); + this.checkVertexExist(sourceV, "source vertex"); + E.checkNotNull(dir, "direction"); + checkPositive(depth, "k-neighbor max_depth"); + checkDegree(degree); + checkLimit(limit); + + Id labelId = this.getEdgeLabelId(label); + + Set latest = newSet(); + latest.add(sourceV); + + Set all = newSet(); + all.add(sourceV); + + while (depth-- > 0) { + long remaining = limit == NO_LIMIT ? NO_LIMIT : limit - all.size(); + latest = this.adjacentVertices(latest, dir, labelId, all, + degree, remaining); + all.addAll(latest); + if (limit != NO_LIMIT && all.size() >= limit) { + break; + } + } + + return all; + } + public Set customizedKneighbor(Id source, EdgeStep step, int maxDepth, long limit) { E.checkNotNull(source, "source vertex id"); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java similarity index 60% rename from hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java rename to hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java index 17a8bce2b7..fddb134411 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedKoutTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java @@ -28,12 +28,69 @@ import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; -public class CustomizedKoutTraverser extends TpTraverser { +public class KoutTraverser extends TpTraverser { - public CustomizedKoutTraverser(HugeGraph graph) { + public KoutTraverser(HugeGraph graph) { super(graph, "kout"); } + public Set kout(Id sourceV, Directions dir, String label, + int depth, boolean nearest, + long degree, long capacity, long limit) { + E.checkNotNull(sourceV, "source vertex id"); + this.checkVertexExist(sourceV, "source vertex"); + E.checkNotNull(dir, "direction"); + checkPositive(depth, "k-out max_depth"); + checkDegree(degree); + checkCapacity(capacity); + checkLimit(limit); + if (capacity != NO_LIMIT) { + // Capacity must > limit because sourceV is counted in capacity + E.checkArgument(capacity >= limit && limit != NO_LIMIT, + "Capacity can't be less than limit, " + + "but got capacity '%s' and limit '%s'", + capacity, limit); + } + + Id labelId = this.getEdgeLabelId(label); + + Set latest = newSet(); + latest.add(sourceV); + + Set all = newSet(); + all.add(sourceV); + + long remaining = capacity == NO_LIMIT ? + NO_LIMIT : capacity - latest.size(); + while (depth-- > 0) { + // Just get limit nodes in last layer if limit < remaining capacity + if (depth == 0 && limit != NO_LIMIT && + (limit < remaining || remaining == NO_LIMIT)) { + remaining = limit; + } + if (nearest) { + latest = this.adjacentVertices(latest, dir, labelId, all, + degree, remaining); + all.addAll(latest); + } else { + latest = this.adjacentVertices(latest, dir, labelId, null, + degree, remaining); + } + if (capacity != NO_LIMIT) { + // Update 'remaining' value to record remaining capacity + remaining -= latest.size(); + + if (remaining <= 0 && depth > 0) { + throw new HugeException( + "Reach capacity '%s' while remaining depth '%s'", + capacity, depth); + } + } + } + + return latest; + } + public Set customizedKout(Id source, EdgeStep step, int maxDepth, boolean nearest, long capacity, long limit) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SameNeighborTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SameNeighborTraverser.java new file mode 100644 index 0000000000..6cb14b6db8 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SameNeighborTraverser.java @@ -0,0 +1,62 @@ +/* + * 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.traversal.algorithm; + +import java.util.Set; + +import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.CollectionUtil; +import com.baidu.hugegraph.util.E; + +public class SameNeighborTraverser extends HugeTraverser { + + public SameNeighborTraverser(HugeGraph graph) { + super(graph); + } + + public Set sameNeighbors(Id vertex, Id other, Directions direction, + String label, long degree, long limit) { + E.checkNotNull(vertex, "vertex id"); + E.checkNotNull(other, "the other vertex id"); + this.checkVertexExist(vertex, "vertex"); + this.checkVertexExist(other, "other vertex"); + E.checkNotNull(direction, "direction"); + checkDegree(degree); + checkLimit(limit); + + Id labelId = this.getEdgeLabelId(label); + + Set sourceNeighbors = IteratorUtils.set(this.adjacentVertices( + vertex, direction, labelId, degree)); + Set targetNeighbors = IteratorUtils.set(this.adjacentVertices( + other, direction, labelId, degree)); + Set sameNeighbors = (Set) CollectionUtil.intersect( + sourceNeighbors, targetNeighbors); + if (limit != NO_LIMIT) { + int end = Math.min(sameNeighbors.size(), (int) limit); + sameNeighbors = CollectionUtil.subSet(sameNeighbors, 0, end); + } + return sameNeighbors; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index 1ec1720351..e4063aabc5 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -25,19 +25,14 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.BiConsumer; -import javax.ws.rs.core.MultivaluedMap; - -import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Vertex; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; -import com.baidu.hugegraph.structure.HugeEdge; import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.strategy.TraverseStrategy; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; @@ -50,7 +45,6 @@ public TemplatePathsTraverser(HugeGraph graph) { super(graph, "template-paths"); } - @SuppressWarnings("unchecked") public Set templatePaths(Iterator sources, Iterator targets, List steps, @@ -80,14 +74,11 @@ public Set templatePaths(Iterator sources, for (RepeatEdgeStep step : steps) { totalSteps += step.maxTimes; } - Traverser traverser = totalSteps >= this.concurrentDepth() ? - new ConcurrentTraverser(sourceList, targetList, - steps, withRing, - capacity, limit) : - new SingleTraverser(sourceList, targetList, - steps, withRing, - capacity, limit); - + TraverseStrategy strategy = this.traverseStrategy( + totalSteps < this.concurrentDepth()); + Traverser traverser = new Traverser(sourceList, targetList, steps, + withRing, capacity, limit, + strategy); do { // Forward traverser.forward(); @@ -103,13 +94,10 @@ public Set templatePaths(Iterator sources, } while (true); } - private abstract class Traverser { + private class Traverser extends PathTraverser { protected final List steps; - protected int stepCount; - protected final long capacity; - protected final long limit; - protected int totalSteps; + protected boolean withRing; protected int sourceIndex; @@ -118,87 +106,37 @@ private abstract class Traverser { protected boolean sourceFinishOneStep = false; protected boolean targetFinishOneStep = false; - protected Map> sources = this.newMultiValueMap(); - protected Map> sourcesAll = this.newMultiValueMap(); - protected Map> targets = this.newMultiValueMap(); - protected Map> targetsAll = this.newMultiValueMap(); - - protected Map> newVertices; - - private Set paths; - public Traverser(Collection sources, Collection targets, List steps, boolean withRing, - long capacity, long limit) { + long capacity, long limit, TraverseStrategy strategy) { + super(sources, targets, capacity, limit, strategy); + this.steps = steps; - this.capacity = capacity; - this.limit = limit; this.withRing = withRing; - - this.stepCount = 0; for (RepeatEdgeStep step : steps) { this.totalSteps += step.maxTimes; } + this.sourceIndex = 0; this.targetIndex = this.steps.size() - 1; - for (Id id : sources) { - this.addNode(this.sources, id, new Node(id)); - } - for (Id id : targets) { - this.addNode(this.targets, id, new Node(id)); - } - this.sourcesAll.putAll(this.sources); - this.targetsAll.putAll(this.targets); - - this.paths = this.newPathSet(); - } - - public void forward() { - RepeatEdgeStep currentStep = this.nextStep(true); - if (currentStep == null) { - return; - } - - this.beforeTraverse(true); - - // Traversal vertices of previous level - traverseOneLayer(this.sources, currentStep, this::forward); - - this.afterTraverse(currentStep, true); - } - - public void backward() { - RepeatEdgeStep currentStep = this.nextStep(false); - if (currentStep == null) { - return; - } - - this.beforeTraverse(false); - - currentStep.swithDirection(); - // Traversal vertices of previous level - traverseOneLayer(this.targets, currentStep, this::backward); - currentStep.swithDirection(); - - this.afterTraverse(currentStep, false); + this.sourceFinishOneStep = false; + this.targetFinishOneStep = false; } public RepeatEdgeStep nextStep(boolean forward) { return forward ? this.forwardStep() : this.backwardStep(); } + @Override public void beforeTraverse(boolean forward) { this.clearNewVertices(); this.reInitAllIfNeeded(forward); } - public abstract void traverseOneLayer( - Map> vertices, - RepeatEdgeStep step, - BiConsumer consumer); + @Override + public void afterTraverse(EdgeStep step, boolean forward) { - public void afterTraverse(RepeatEdgeStep step, boolean forward) { Map> all = forward ? this.sourcesAll : this.targetsAll; this.addNewVerticesToAll(all); @@ -206,38 +144,8 @@ public void afterTraverse(RepeatEdgeStep step, boolean forward) { this.stepCount++; } - private void forward(Id v, RepeatEdgeStep step) { - this.traverseOne(v, step, true); - } - - private void backward(Id v, RepeatEdgeStep step) { - this.traverseOne(v, step, false); - } - - private void traverseOne(Id source, RepeatEdgeStep step, - boolean forward) { - if (this.reachLimit()) { - return; - } - - Iterator edges = edgesOfVertex(source, step); - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - this.processOne(source, target, forward); - } - } - - private void processOne(Id source, Id target, boolean forward) { - if (forward) { - processOneForForward(source, target); - } else { - processOneForBackward(source, target); - } - } - - private void processOneForForward(Id sourceV, Id targetV) { + @Override + protected void processOneForForward(Id sourceV, Id targetV) { for (Node source : this.sources.get(sourceV)) { // If have loop, skip target if (!this.withRing && source.contains(targetV)) { @@ -263,7 +171,8 @@ private void processOneForForward(Id sourceV, Id targetV) { } } - private void processOneForBackward(Id sourceV, Id targetV) { + @Override + protected void processOneForBackward(Id sourceV, Id targetV) { for (Node source : this.targets.get(sourceV)) { // If have loop, skip target if (!this.withRing && source.contains(targetV)) { @@ -313,12 +222,14 @@ private void reInitAllIfNeeded(boolean forward) { } } - private void reInitCurrentStepIfNeeded(RepeatEdgeStep step, - boolean forward) { - step.decreaseTimes(); + @Override + protected void reInitCurrentStepIfNeeded(EdgeStep step, + boolean forward) { + RepeatEdgeStep currentStep = (RepeatEdgeStep) step; + currentStep.decreaseTimes(); if (forward) { // Re-init sources - if (step.remainTimes() > 0) { + if (currentStep.remainTimes() > 0) { this.sources = this.newVertices; } else { this.sources = this.sourcesAll; @@ -326,7 +237,7 @@ private void reInitCurrentStepIfNeeded(RepeatEdgeStep step, } } else { // Re-init targets - if (step.remainTimes() > 0) { + if (currentStep.remainTimes() > 0) { this.targets = this.newVertices; } else { this.targets = this.targetsAll; @@ -367,141 +278,6 @@ public boolean lastSuperStep() { return this.targetIndex == this.sourceIndex || this.targetIndex == this.sourceIndex + 1; } - - public void clearNewVertices() { - this.newVertices = this.newMultiValueMap(); - } - - public void addNodeToNewVertices(Id id, Node node) { - this.addNode(this.newVertices, id, node); - } - - public abstract Map> newMultiValueMap(); - - public abstract Set newPathSet(); - - public abstract void addNode(Map> vertices, - Id id, Node node); - - public abstract void addNewVerticesToAll(Map> targets); - - public Set paths() { - return this.paths; - } - - public int pathCount() { - return this.paths.size(); - } - - protected boolean finish() { - return this.stepCount >= this.totalSteps || this.reachLimit(); - } - - protected boolean reachLimit() { - checkCapacity(this.capacity, this.accessedNodes(), - "template paths"); - if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { - return false; - } - return true; - } - - private int accessedNodes() { - int size = 0; - for (List value : this.sourcesAll.values()) { - size += value.size(); - } - for (List value : this.targetsAll.values()) { - size += value.size(); - } - return size; - } - } - - private class ConcurrentTraverser extends Traverser { - - public ConcurrentTraverser(Collection sources, - Collection targets, - List steps, - boolean withRing, - long capacity, long limit) { - super(sources, targets, steps, withRing, capacity, limit); - } - - @Override - public Map> newMultiValueMap() { - return new ConcurrentMultiValuedMap<>(); - } - - @Override - public void traverseOneLayer( - Map> vertices, RepeatEdgeStep step, - BiConsumer consumer) { - traverseIds(this.sources.keySet().iterator(), id -> { - consumer.accept(id, step); - }); - } - - @Override - public Set newPathSet() { - return ConcurrentHashMap.newKeySet(); - } - - @Override - public void addNode(Map> vertices, Id id, Node node) { - ((ConcurrentMultiValuedMap) vertices).add(id, node); - } - - @Override - public void addNewVerticesToAll(Map> targets) { - ConcurrentMultiValuedMap vertices = - (ConcurrentMultiValuedMap) targets; - for (Map.Entry> entry : this.newVertices.entrySet()) { - vertices.addAll(entry.getKey(), entry.getValue()); - } - } - } - - private class SingleTraverser extends Traverser { - - public SingleTraverser(Collection sources, Collection targets, - List steps, boolean withRing, - long capacity, long limit) { - super(sources, targets, steps, withRing, capacity, limit); - } - - @Override - public Map> newMultiValueMap() { - return newMultivalueMap(); - } - - @Override - public Set newPathSet() { - return new PathSet(); - } - - @Override - public void traverseOneLayer( - Map> vertices, RepeatEdgeStep step, - BiConsumer consumer) { - for (Id id : vertices.keySet()) { - consumer.accept(id, step); - } - } - - @Override - public void addNode(Map> vertices, Id id, Node node) { - ((MultivaluedMap) vertices).add(id, node); - } - - @Override - public void addNewVerticesToAll(Map> targets) { - MultivaluedMap vertices = - (MultivaluedMap) targets; - for (Map.Entry> entry : this.newVertices.entrySet()) { - vertices.addAll(entry.getKey(), entry.getValue()); - } - } } public static class RepeatEdgeStep extends EdgeStep { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index e644e3a781..3a0b07e462 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -19,13 +19,16 @@ package com.baidu.hugegraph.traversal.algorithm; +import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; import java.util.function.Consumer; import org.apache.commons.lang3.tuple.Pair; @@ -41,6 +44,9 @@ import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.iterator.FilterIterator; import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.traversal.algorithm.strategy.ConcurrentTraverseStrategy; +import com.baidu.hugegraph.traversal.algorithm.strategy.SingleTraverseStrategy; +import com.baidu.hugegraph.traversal.algorithm.strategy.TraverseStrategy; import com.baidu.hugegraph.util.Consumers; import jersey.repackaged.com.google.common.base.Objects; @@ -171,6 +177,199 @@ protected boolean match(Element elem, String key, Object value) { return p.isPresent() && Objects.equal(p.value(), value); } + protected TraverseStrategy traverseStrategy(boolean single) { + return single ? new SingleTraverseStrategy(this.graph()) : + new ConcurrentTraverseStrategy(this.graph()); + + } + + public abstract class PathTraverser { + + protected int stepCount; + protected final long capacity; + protected final long limit; + protected int totalSteps; // TODO: delete or implement abstract method + + protected Map> sources; + protected Map> sourcesAll; + protected Map> targets; + protected Map> targetsAll; + + protected Map> newVertices; + + protected Set paths; + + protected TraverseStrategy traverseStrategy; + + public PathTraverser(Collection sources, Collection targets, + long capacity, long limit, + TraverseStrategy strategy) { + this.traverseStrategy = strategy; + + this.capacity = capacity; + this.limit = limit; + + this.stepCount = 0; + + this.sources = this.newMultiValueMap(); + this.sourcesAll = this.newMultiValueMap(); + this.targets = this.newMultiValueMap(); + this.targetsAll = this.newMultiValueMap(); + + for (Id id : sources) { + this.addNode(this.sources, id, new Node(id)); + } + for (Id id : targets) { + this.addNode(this.targets, id, new Node(id)); + } + this.sourcesAll.putAll(this.sources); + this.targetsAll.putAll(this.targets); + + this.paths = this.newPathSet(); + } + + public void forward() { + EdgeStep currentStep = this.nextStep(true); + if (currentStep == null) { + return; + } + + this.beforeTraverse(true); + + // Traversal vertices of previous level + traverseOneLayer(this.sources, currentStep, this::forward); + + this.afterTraverse(currentStep, true); + } + + public void backward() { + EdgeStep currentStep = this.nextStep(false); + if (currentStep == null) { + return; + } + + this.beforeTraverse(false); + + currentStep.swithDirection(); + // Traversal vertices of previous level + traverseOneLayer(this.targets, currentStep, this::backward); + currentStep.swithDirection(); + + this.afterTraverse(currentStep, false); + } + + public abstract EdgeStep nextStep(boolean forward); + + public void beforeTraverse(boolean forward) { + this.clearNewVertices(); + } + + public void traverseOneLayer(Map> vertices, + EdgeStep step, + BiConsumer consumer) { + this.traverseStrategy.traverseOneLayer(vertices, step, consumer); + } + + public void afterTraverse(EdgeStep step, boolean forward) { + this.reInitCurrentStepIfNeeded(step, forward); + this.stepCount++; + } + + private void forward(Id v, EdgeStep step) { + this.traverseOne(v, step, true); + } + + private void backward(Id v, EdgeStep step) { + this.traverseOne(v, step, false); + } + + private void traverseOne(Id v, EdgeStep step, boolean forward) { + if (this.reachLimit()) { + return; + } + + Iterator edges = edgesOfVertex(v, step); + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + this.processOne(v, target, forward); + } + } + + private void processOne(Id source, Id target, boolean forward) { + if (forward) { + processOneForForward(source, target); + } else { + processOneForBackward(source, target); + } + } + + protected abstract void processOneForForward(Id source, Id target); + + protected abstract void processOneForBackward(Id source, Id target); + + protected abstract void reInitCurrentStepIfNeeded(EdgeStep step, + boolean forward); + + public void clearNewVertices() { + this.newVertices = this.newMultiValueMap(); + } + + public void addNodeToNewVertices(Id id, Node node) { + this.addNode(this.newVertices, id, node); + } + + public Map> newMultiValueMap() { + return this.traverseStrategy.newMultiValueMap(); + } + + public Set newPathSet() { + return this.traverseStrategy.newPathSet(); + } + + public void addNode(Map> vertices, Id id, Node node) { + this.traverseStrategy.addNode(vertices, id, node); + } + + public void addNewVerticesToAll(Map> targets) { + this.traverseStrategy.addNewVerticesToAll(this.newVertices, + targets); + } + + public Set paths() { + return this.paths; + } + + public int pathCount() { + return this.paths.size(); + } + + protected boolean finish() { + return this.stepCount >= this.totalSteps || this.reachLimit(); + } + + protected boolean reachLimit() { + HugeTraverser.checkCapacity(this.capacity, this.accessedNodes(), + "template paths"); + if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { + return false; + } + return true; + } + + protected int accessedNodes() { + int size = 0; + for (List value : this.sourcesAll.values()) { + size += value.size(); + } + for (List value : this.targetsAll.values()) { + size += value.size(); + } + return size; + } + } + public class ConcurrentMultiValuedMap extends ConcurrentHashMap> { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java new file mode 100644 index 0000000000..9d20259516 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java @@ -0,0 +1,74 @@ +/* + * 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.traversal.algorithm.strategy; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.BiConsumer; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.TpTraverser; + +public class ConcurrentTraverseStrategy extends TpTraverser + implements TraverseStrategy { + + public ConcurrentTraverseStrategy(HugeGraph graph) { + super(graph); + } + + @Override + public Map> newMultiValueMap() { + return new TpTraverser.ConcurrentMultiValuedMap<>(); + } + + @Override + public void traverseOneLayer(Map> vertices, + EdgeStep step, + BiConsumer biConsumer) { + traverseIds(vertices.keySet().iterator(), (id) -> { + biConsumer.accept(id, step); + }); + } + + @Override + public Set newPathSet() { + return ConcurrentHashMap.newKeySet(); + } + + @Override + public void addNode(Map> vertices, + Id id, Node node) { + ((ConcurrentMultiValuedMap) vertices).add(id, node); + } + + @Override + public void addNewVerticesToAll(Map> newVertices, + Map> targets) { + ConcurrentMultiValuedMap vertices = + (ConcurrentMultiValuedMap) targets; + for (Map.Entry> entry : newVertices.entrySet()) { + vertices.addAll(entry.getKey(), entry.getValue()); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java new file mode 100644 index 0000000000..75401d9768 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java @@ -0,0 +1,75 @@ +/* + * 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.traversal.algorithm.strategy; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiConsumer; + +import javax.ws.rs.core.MultivaluedMap; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; +import com.baidu.hugegraph.traversal.algorithm.TpTraverser; + +public class SingleTraverseStrategy extends TpTraverser + implements TraverseStrategy { + + public SingleTraverseStrategy(HugeGraph graph) { + super(graph); + } + + @Override + public void traverseOneLayer(Map> vertices, + EdgeStep step, + BiConsumer biConsumer) { + for (Id id : vertices.keySet()) { + biConsumer.accept(id, step); + } + } + + @Override + public Map> newMultiValueMap() { + return newMultivalueMap(); + } + + @Override + public Set newPathSet() { + return new HugeTraverser.PathSet(); + } + + @Override + public void addNode(Map> vertices, Id id, Node node) { + ((MultivaluedMap) vertices).add(id, node); + } + + @Override + public void addNewVerticesToAll(Map> newVertices, + Map> targets) { + MultivaluedMap vertices = + (MultivaluedMap) targets; + for (Map.Entry> entry : newVertices.entrySet()) { + vertices.addAll(entry.getKey(), entry.getValue()); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java new file mode 100644 index 0000000000..339b888549 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java @@ -0,0 +1,47 @@ +/* + * 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.traversal.algorithm.strategy; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiConsumer; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.EdgeStep; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; + +public interface TraverseStrategy { + + public abstract void traverseOneLayer( + Map> vertices, + EdgeStep step, BiConsumer consumer); + + public abstract Map> newMultiValueMap(); + + public abstract Set newPathSet(); + + public abstract void addNode(Map> vertices, + Id id, HugeTraverser.Node node); + + public abstract void addNewVerticesToAll( + Map> newVertices, + Map> targets); +} From 9e8372a3adb05d109ed1c640a0087016ab0ad571 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Wed, 14 Oct 2020 11:12:31 +0800 Subject: [PATCH 18/20] improve Change-Id: I760194b0f02a41873c2157a5ddf59dd4d1c3ceff --- .../algorithm/CollectionPathsTraverser.java | 40 +--- .../traversal/algorithm/HugeTraverser.java | 10 +- .../traversal/algorithm/PathTraverser.java | 226 ++++++++++++++++++ .../algorithm/TemplatePathsTraverser.java | 2 +- .../traversal/algorithm/TpTraverser.java | 193 +-------------- .../strategy/ConcurrentTraverseStrategy.java | 8 + .../strategy/SingleTraverseStrategy.java | 8 + .../algorithm/strategy/TraverseStrategy.java | 5 + 8 files changed, 269 insertions(+), 223 deletions(-) create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java index 2f5a0c23b4..3ae950c2c2 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -25,16 +25,12 @@ import java.util.List; import java.util.Map; -import javax.ws.rs.core.MultivaluedMap; - import org.apache.tinkerpop.gremlin.structure.Vertex; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.structure.HugeVertex; -import com.baidu.hugegraph.traversal.algorithm.strategy.ConcurrentTraverseStrategy; -import com.baidu.hugegraph.traversal.algorithm.strategy.SingleTraverseStrategy; import com.baidu.hugegraph.traversal.algorithm.strategy.TraverseStrategy; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; @@ -74,22 +70,14 @@ public Collection paths(Iterator sources, checkPositive(depth, "max depth"); Traverser traverser; - - TraverseStrategy strategy; - if (depth >= this.concurrentDepth()) { - strategy = new ConcurrentTraverseStrategy(this.graph()); - traverser = new Traverser(sourceList, targetList, step, depth, - capacity, limit, strategy); + TraverseStrategy strategy = this.traverseStrategy( + depth >= this.concurrentDepth()); + if (nearest) { + traverser = new NearestTraverser(sourceList, targetList, step, + depth, capacity, limit, strategy); } else { - strategy = new SingleTraverseStrategy(this.graph()); - if (nearest) { - traverser = new SingleNearestTraverser(sourceList, targetList, - step, depth, capacity, - limit, strategy); - } else { - traverser = new Traverser(sourceList, targetList, step, - depth, capacity, limit, strategy); - } + traverser = new Traverser(sourceList, targetList, step, + depth, capacity, limit, strategy); } do { @@ -195,12 +183,12 @@ protected void reInitCurrentStepIfNeeded(EdgeStep step, } } - private class SingleNearestTraverser extends Traverser { + private class NearestTraverser extends Traverser { - public SingleNearestTraverser(Collection sources, - Collection targets, EdgeStep step, - int depth, long capacity, long limit, - TraverseStrategy strategy) { + public NearestTraverser(Collection sources, + Collection targets, EdgeStep step, + int depth, long capacity, long limit, + TraverseStrategy strategy) { super(sources, targets, step, depth, capacity, limit, strategy); } @@ -277,10 +265,8 @@ public void addNodeToNewVertices(Id id, Node node) { @Override public void addNewVerticesToAll(Map> targets) { - MultivaluedMap vertices = - (MultivaluedMap) targets; for (Map.Entry> entry : this.newVertices.entrySet()) { - vertices.putIfAbsent(entry.getKey(), entry.getValue()); + targets.putIfAbsent(entry.getKey(), entry.getValue()); } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java index 3ef436eebe..c3ed9cbd18 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java @@ -403,14 +403,14 @@ public static Iterator skipSuperNodeIfNeeded(Iterator edges, } protected static Set newSet() { - return newSet(true); + return newSet(false); } - protected static Set newSet(boolean single) { - if (single) { - return new HashSet<>(); - } else { + protected static Set newSet(boolean concurrent) { + if (concurrent) { return ConcurrentHashMap.newKeySet(); + } else { + return new HashSet<>(); } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java new file mode 100644 index 0000000000..6d7702535a --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java @@ -0,0 +1,226 @@ +/* + * 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.traversal.algorithm; + +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiConsumer; + +import org.apache.tinkerpop.gremlin.structure.Edge; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.traversal.algorithm.strategy.TraverseStrategy; + +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.NO_LIMIT; + +public abstract class PathTraverser { + + protected int stepCount; + protected final long capacity; + protected final long limit; + protected int totalSteps; // TODO: delete or implement abstract method + + protected Map> sources; + protected Map> sourcesAll; + protected Map> targets; + protected Map> targetsAll; + + protected Map> newVertices; + + protected Set paths; + + protected TraverseStrategy traverseStrategy; + + public PathTraverser(Collection sources, Collection targets, + long capacity, long limit, + TraverseStrategy strategy) { + this.traverseStrategy = strategy; + + this.capacity = capacity; + this.limit = limit; + + this.stepCount = 0; + + this.sources = this.newMultiValueMap(); + this.sourcesAll = this.newMultiValueMap(); + this.targets = this.newMultiValueMap(); + this.targetsAll = this.newMultiValueMap(); + + for (Id id : sources) { + this.addNode(this.sources, id, new HugeTraverser.Node(id)); + } + for (Id id : targets) { + this.addNode(this.targets, id, new HugeTraverser.Node(id)); + } + this.sourcesAll.putAll(this.sources); + this.targetsAll.putAll(this.targets); + + this.paths = this.newPathSet(); + } + + public void forward() { + EdgeStep currentStep = this.nextStep(true); + if (currentStep == null) { + return; + } + + this.beforeTraverse(true); + + // Traversal vertices of previous level + traverseOneLayer(this.sources, currentStep, this::forward); + + this.afterTraverse(currentStep, true); + } + + public void backward() { + EdgeStep currentStep = this.nextStep(false); + if (currentStep == null) { + return; + } + + this.beforeTraverse(false); + + currentStep.swithDirection(); + // Traversal vertices of previous level + traverseOneLayer(this.targets, currentStep, this::backward); + currentStep.swithDirection(); + + this.afterTraverse(currentStep, false); + } + + public abstract EdgeStep nextStep(boolean forward); + + public void beforeTraverse(boolean forward) { + this.clearNewVertices(); + } + + public void traverseOneLayer(Map> vertices, + EdgeStep step, + BiConsumer consumer) { + this.traverseStrategy.traverseOneLayer(vertices, step, consumer); + } + + public void afterTraverse(EdgeStep step, boolean forward) { + this.reInitCurrentStepIfNeeded(step, forward); + this.stepCount++; + } + + private void forward(Id v, EdgeStep step) { + this.traverseOne(v, step, true); + } + + private void backward(Id v, EdgeStep step) { + this.traverseOne(v, step, false); + } + + private void traverseOne(Id v, EdgeStep step, boolean forward) { + if (this.reachLimit()) { + return; + } + + Iterator edges = this.edgesOfVertex(v, step); + while (edges.hasNext()) { + HugeEdge edge = (HugeEdge) edges.next(); + Id target = edge.id().otherVertexId(); + + this.processOne(v, target, forward); + } + } + + protected Iterator edgesOfVertex(Id source, EdgeStep edgeStep) { + return this.traverseStrategy.edgesOfVertex(source, edgeStep); + } + + private void processOne(Id source, Id target, boolean forward) { + if (forward) { + processOneForForward(source, target); + } else { + processOneForBackward(source, target); + } + } + + protected abstract void processOneForForward(Id source, Id target); + + protected abstract void processOneForBackward(Id source, Id target); + + protected abstract void reInitCurrentStepIfNeeded(EdgeStep step, + boolean forward); + + public void clearNewVertices() { + this.newVertices = this.newMultiValueMap(); + } + + public void addNodeToNewVertices(Id id, HugeTraverser.Node node) { + this.addNode(this.newVertices, id, node); + } + + public Map> newMultiValueMap() { + return this.traverseStrategy.newMultiValueMap(); + } + + public Set newPathSet() { + return this.traverseStrategy.newPathSet(); + } + + public void addNode(Map> vertices, Id id, + HugeTraverser.Node node) { + this.traverseStrategy.addNode(vertices, id, node); + } + + public void addNewVerticesToAll(Map> targets) { + this.traverseStrategy.addNewVerticesToAll(this.newVertices, targets); + } + + public Set paths() { + return this.paths; + } + + public int pathCount() { + return this.paths.size(); + } + + protected boolean finish() { + return this.stepCount >= this.totalSteps || this.reachLimit(); + } + + protected boolean reachLimit() { + HugeTraverser.checkCapacity(this.capacity, this.accessedNodes(), + "template paths"); + if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { + return false; + } + return true; + } + + protected int accessedNodes() { + int size = 0; + for (List value : this.sourcesAll.values()) { + size += value.size(); + } + for (List value : this.targetsAll.values()) { + size += value.size(); + } + return size; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index e4063aabc5..ec6b2afef0 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -75,7 +75,7 @@ public Set templatePaths(Iterator sources, totalSteps += step.maxTimes; } TraverseStrategy strategy = this.traverseStrategy( - totalSteps < this.concurrentDepth()); + totalSteps >= this.concurrentDepth()); Traverser traverser = new Traverser(sourceList, targetList, steps, withRing, capacity, limit, strategy); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index 3a0b07e462..62429b6ee3 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -177,199 +177,12 @@ protected boolean match(Element elem, String key, Object value) { return p.isPresent() && Objects.equal(p.value(), value); } - protected TraverseStrategy traverseStrategy(boolean single) { - return single ? new SingleTraverseStrategy(this.graph()) : - new ConcurrentTraverseStrategy(this.graph()); + protected TraverseStrategy traverseStrategy(boolean concurrent) { + return concurrent ? new ConcurrentTraverseStrategy(this.graph()) : + new SingleTraverseStrategy(this.graph()); } - public abstract class PathTraverser { - - protected int stepCount; - protected final long capacity; - protected final long limit; - protected int totalSteps; // TODO: delete or implement abstract method - - protected Map> sources; - protected Map> sourcesAll; - protected Map> targets; - protected Map> targetsAll; - - protected Map> newVertices; - - protected Set paths; - - protected TraverseStrategy traverseStrategy; - - public PathTraverser(Collection sources, Collection targets, - long capacity, long limit, - TraverseStrategy strategy) { - this.traverseStrategy = strategy; - - this.capacity = capacity; - this.limit = limit; - - this.stepCount = 0; - - this.sources = this.newMultiValueMap(); - this.sourcesAll = this.newMultiValueMap(); - this.targets = this.newMultiValueMap(); - this.targetsAll = this.newMultiValueMap(); - - for (Id id : sources) { - this.addNode(this.sources, id, new Node(id)); - } - for (Id id : targets) { - this.addNode(this.targets, id, new Node(id)); - } - this.sourcesAll.putAll(this.sources); - this.targetsAll.putAll(this.targets); - - this.paths = this.newPathSet(); - } - - public void forward() { - EdgeStep currentStep = this.nextStep(true); - if (currentStep == null) { - return; - } - - this.beforeTraverse(true); - - // Traversal vertices of previous level - traverseOneLayer(this.sources, currentStep, this::forward); - - this.afterTraverse(currentStep, true); - } - - public void backward() { - EdgeStep currentStep = this.nextStep(false); - if (currentStep == null) { - return; - } - - this.beforeTraverse(false); - - currentStep.swithDirection(); - // Traversal vertices of previous level - traverseOneLayer(this.targets, currentStep, this::backward); - currentStep.swithDirection(); - - this.afterTraverse(currentStep, false); - } - - public abstract EdgeStep nextStep(boolean forward); - - public void beforeTraverse(boolean forward) { - this.clearNewVertices(); - } - - public void traverseOneLayer(Map> vertices, - EdgeStep step, - BiConsumer consumer) { - this.traverseStrategy.traverseOneLayer(vertices, step, consumer); - } - - public void afterTraverse(EdgeStep step, boolean forward) { - this.reInitCurrentStepIfNeeded(step, forward); - this.stepCount++; - } - - private void forward(Id v, EdgeStep step) { - this.traverseOne(v, step, true); - } - - private void backward(Id v, EdgeStep step) { - this.traverseOne(v, step, false); - } - - private void traverseOne(Id v, EdgeStep step, boolean forward) { - if (this.reachLimit()) { - return; - } - - Iterator edges = edgesOfVertex(v, step); - while (edges.hasNext()) { - HugeEdge edge = (HugeEdge) edges.next(); - Id target = edge.id().otherVertexId(); - - this.processOne(v, target, forward); - } - } - - private void processOne(Id source, Id target, boolean forward) { - if (forward) { - processOneForForward(source, target); - } else { - processOneForBackward(source, target); - } - } - - protected abstract void processOneForForward(Id source, Id target); - - protected abstract void processOneForBackward(Id source, Id target); - - protected abstract void reInitCurrentStepIfNeeded(EdgeStep step, - boolean forward); - - public void clearNewVertices() { - this.newVertices = this.newMultiValueMap(); - } - - public void addNodeToNewVertices(Id id, Node node) { - this.addNode(this.newVertices, id, node); - } - - public Map> newMultiValueMap() { - return this.traverseStrategy.newMultiValueMap(); - } - - public Set newPathSet() { - return this.traverseStrategy.newPathSet(); - } - - public void addNode(Map> vertices, Id id, Node node) { - this.traverseStrategy.addNode(vertices, id, node); - } - - public void addNewVerticesToAll(Map> targets) { - this.traverseStrategy.addNewVerticesToAll(this.newVertices, - targets); - } - - public Set paths() { - return this.paths; - } - - public int pathCount() { - return this.paths.size(); - } - - protected boolean finish() { - return this.stepCount >= this.totalSteps || this.reachLimit(); - } - - protected boolean reachLimit() { - HugeTraverser.checkCapacity(this.capacity, this.accessedNodes(), - "template paths"); - if (this.limit == NO_LIMIT || this.pathCount() < this.limit) { - return false; - } - return true; - } - - protected int accessedNodes() { - int size = 0; - for (List value : this.sourcesAll.values()) { - size += value.size(); - } - for (List value : this.targetsAll.values()) { - size += value.size(); - } - return size; - } - } - public class ConcurrentMultiValuedMap extends ConcurrentHashMap> { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java index 9d20259516..fa48fb5c80 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java @@ -19,12 +19,15 @@ package com.baidu.hugegraph.traversal.algorithm.strategy; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiConsumer; +import org.apache.tinkerpop.gremlin.structure.Edge; + import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; @@ -71,4 +74,9 @@ public void addNewVerticesToAll(Map> newVertices, vertices.addAll(entry.getKey(), entry.getValue()); } } + + @Override + public Iterator edgesOfVertex(Id source, EdgeStep edgeStep) { + return super.edgesOfVertex(source, edgeStep); + } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java index 75401d9768..44b5bdae84 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java @@ -19,6 +19,7 @@ package com.baidu.hugegraph.traversal.algorithm.strategy; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -26,6 +27,8 @@ import javax.ws.rs.core.MultivaluedMap; +import org.apache.tinkerpop.gremlin.structure.Edge; + import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; @@ -72,4 +75,9 @@ public void addNewVerticesToAll(Map> newVertices, vertices.addAll(entry.getKey(), entry.getValue()); } } + + @Override + public Iterator edgesOfVertex(Id source, EdgeStep edgeStep) { + return super.edgesOfVertex(source, edgeStep); + } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java index 339b888549..93a73b3068 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java @@ -19,11 +19,14 @@ package com.baidu.hugegraph.traversal.algorithm.strategy; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; +import org.apache.tinkerpop.gremlin.structure.Edge; + import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; @@ -44,4 +47,6 @@ public abstract void addNode(Map> vertices, public abstract void addNewVerticesToAll( Map> newVertices, Map> targets); + + public Iterator edgesOfVertex(Id source, EdgeStep edgeStep); } From 4fd93f95cdac548c3d56f431d753a03e5ec6da69 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Tue, 20 Oct 2020 21:16:57 +0800 Subject: [PATCH 19/20] normalize traverser Change-Id: I68226c55ca387ecfdd103e4af8b5e45faa9a934e --- .../java/com/baidu/hugegraph/HugeFactory.java | 2 + .../algorithm/CollectionPathsTraverser.java | 46 +++++++++++-------- .../traversal/algorithm/HugeTraverser.java | 10 ++++ .../algorithm/JaccardSimilarTraverser.java | 2 +- .../algorithm/KneighborTraverser.java | 2 +- .../traversal/algorithm/KoutTraverser.java | 2 +- .../MultiNodeShortestPathTraverser.java | 2 +- .../traversal/algorithm/PathTraverser.java | 25 +++++----- .../algorithm/TemplatePathsTraverser.java | 28 +++++------ .../traversal/algorithm/TpTraverser.java | 37 +++------------ .../strategy/ConcurrentTraverseStrategy.java | 8 ---- .../strategy/SingleTraverseStrategy.java | 10 +--- .../algorithm/strategy/TraverseStrategy.java | 10 ++-- 13 files changed, 83 insertions(+), 101 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java index d02e3a4e16..c2b6f0c7a2 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java @@ -34,6 +34,7 @@ import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.event.EventHub; import com.baidu.hugegraph.task.TaskManager; +import com.baidu.hugegraph.traversal.algorithm.TpTraverser; import com.baidu.hugegraph.type.define.SerialEnum; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; @@ -127,6 +128,7 @@ public static void shutdown(long timeout) { throw new TimeoutException(timeout + "s"); } TaskManager.instance().shutdown(timeout); + TpTraverser.destroy(); } catch (Throwable e) { LOG.error("Error while shutdown", e); throw new HugeException("Failed to shutdown", e); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java index 3ae950c2c2..7da8ea0e51 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -36,12 +36,12 @@ import com.baidu.hugegraph.util.Log; import com.google.common.collect.ImmutableList; -public class CollectionPathsTraverser extends TpTraverser { +public class CollectionPathsTraverser extends HugeTraverser { private static final Logger LOG = Log.logger(CollectionPathsTraverser.class); public CollectionPathsTraverser(HugeGraph graph) { - super(graph, "collection-paths"); + super(graph); } public Collection paths(Iterator sources, @@ -69,40 +69,44 @@ public Collection paths(Iterator sources, "but got: %s", MAX_VERTICES, sourceList.size()); checkPositive(depth, "max depth"); + + TraverseStrategy strategy = TraverseStrategy.create( + depth >= this.concurrentDepth(), + this.graph()); Traverser traverser; - TraverseStrategy strategy = this.traverseStrategy( - depth >= this.concurrentDepth()); if (nearest) { - traverser = new NearestTraverser(sourceList, targetList, step, - depth, capacity, limit, strategy); + traverser = new NearestTraverser(this, strategy, + sourceList, targetList, step, + depth, capacity, limit); } else { - traverser = new Traverser(sourceList, targetList, step, - depth, capacity, limit, strategy); + traverser = new Traverser(this, strategy, + sourceList, targetList, step, + depth, capacity, limit); } do { // Forward traverser.forward(); - if (traverser.finish()) { + if (traverser.finished()) { return traverser.paths(); } // Backward traverser.backward(); - if (traverser.finish()) { + if (traverser.finished()) { return traverser.paths(); } } while (true); } - private class Traverser extends PathTraverser { + private static class Traverser extends PathTraverser { protected final EdgeStep step; - public Traverser(Collection sources, Collection targets, - EdgeStep step, int depth, long capacity, - long limit, TraverseStrategy strategy) { - super(sources, targets, capacity, limit, strategy); + public Traverser(HugeTraverser traverser, TraverseStrategy strategy, + Collection sources, Collection targets, + EdgeStep step, int depth, long capacity, long limit) { + super(traverser, strategy, sources, targets, capacity, limit); this.step = step; this.totalSteps = depth; } @@ -185,11 +189,13 @@ protected void reInitCurrentStepIfNeeded(EdgeStep step, private class NearestTraverser extends Traverser { - public NearestTraverser(Collection sources, - Collection targets, EdgeStep step, - int depth, long capacity, long limit, - TraverseStrategy strategy) { - super(sources, targets, step, depth, capacity, limit, strategy); + public NearestTraverser(HugeTraverser traverser, + TraverseStrategy strategy, + Collection sources, Collection targets, + EdgeStep step, int depth, long capacity, + long limit) { + super(traverser, strategy, sources, targets, step, + depth, capacity, limit); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java index c3ed9cbd18..8f6102f9ae 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/HugeTraverser.java @@ -44,6 +44,8 @@ import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.query.QueryResults; import com.baidu.hugegraph.backend.tx.GraphTransaction; +import com.baidu.hugegraph.config.CoreOptions; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.exception.NotFoundException; import com.baidu.hugegraph.iterator.ExtendableIterator; import com.baidu.hugegraph.iterator.FilterIterator; @@ -90,6 +92,14 @@ public HugeGraph graph() { return this.graph; } + protected int concurrentDepth() { + return this.config().get(CoreOptions.OLTP_CONCURRENT_DEPTH); + } + + protected HugeConfig config() { + return ((HugeConfig) this.graph().hugegraph().configuration()); + } + protected Set adjacentVertices(Set vertices, Directions dir, Id label, Set excluded, long degree, long limit) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java index 321de5b064..70dd1b6a4f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java @@ -39,7 +39,7 @@ public class JaccardSimilarTraverser extends TpTraverser { public JaccardSimilarTraverser(HugeGraph graph) { - super(graph, "jaccard-similar"); + super(graph); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java index 2210653661..7ab46cbd47 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java @@ -30,7 +30,7 @@ public class KneighborTraverser extends TpTraverser { public KneighborTraverser(HugeGraph graph) { - super(graph, "kneighbor"); + super(graph); } public Set kneighbor(Id sourceV, Directions dir, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java index fddb134411..c225065d17 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java @@ -31,7 +31,7 @@ public class KoutTraverser extends TpTraverser { public KoutTraverser(HugeGraph graph) { - super(graph, "kout"); + super(graph); } public Set kout(Id sourceV, Directions dir, String label, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java index 63fd7c7149..26d5fab2b9 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java @@ -40,7 +40,7 @@ public class MultiNodeShortestPathTraverser extends TpTraverser { public MultiNodeShortestPathTraverser(HugeGraph graph) { - super(graph, "multi-node-shortest-path"); + super(graph); } public List multiNodeShortestPath(Iterator vertices, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java index 6d7702535a..9551009e5e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PathTraverser.java @@ -36,6 +36,8 @@ public abstract class PathTraverser { + protected final HugeTraverser traverser; + protected int stepCount; protected final long capacity; protected final long limit; @@ -52,9 +54,10 @@ public abstract class PathTraverser { protected TraverseStrategy traverseStrategy; - public PathTraverser(Collection sources, Collection targets, - long capacity, long limit, - TraverseStrategy strategy) { + public PathTraverser(HugeTraverser traverser, TraverseStrategy strategy, + Collection sources, Collection targets, + long capacity, long limit) { + this.traverser = traverser; this.traverseStrategy = strategy; this.capacity = capacity; @@ -88,7 +91,7 @@ public void forward() { this.beforeTraverse(true); // Traversal vertices of previous level - traverseOneLayer(this.sources, currentStep, this::forward); + this.traverseOneLayer(this.sources, currentStep, this::forward); this.afterTraverse(currentStep, true); } @@ -103,7 +106,7 @@ public void backward() { currentStep.swithDirection(); // Traversal vertices of previous level - traverseOneLayer(this.targets, currentStep, this::backward); + this.traverseOneLayer(this.targets, currentStep, this::backward); currentStep.swithDirection(); this.afterTraverse(currentStep, false); @@ -139,7 +142,7 @@ private void traverseOne(Id v, EdgeStep step, boolean forward) { return; } - Iterator edges = this.edgesOfVertex(v, step); + Iterator edges = this.traverser.edgesOfVertex(v, step); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); Id target = edge.id().otherVertexId(); @@ -148,15 +151,11 @@ private void traverseOne(Id v, EdgeStep step, boolean forward) { } } - protected Iterator edgesOfVertex(Id source, EdgeStep edgeStep) { - return this.traverseStrategy.edgesOfVertex(source, edgeStep); - } - private void processOne(Id source, Id target, boolean forward) { if (forward) { - processOneForForward(source, target); + this.processOneForForward(source, target); } else { - processOneForBackward(source, target); + this.processOneForBackward(source, target); } } @@ -200,7 +199,7 @@ public int pathCount() { return this.paths.size(); } - protected boolean finish() { + protected boolean finished() { return this.stepCount >= this.totalSteps || this.reachLimit(); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java index ec6b2afef0..d22c01d32f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TemplatePathsTraverser.java @@ -37,12 +37,12 @@ import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; -public class TemplatePathsTraverser extends TpTraverser { +public class TemplatePathsTraverser extends HugeTraverser { private static final Logger LOG = Log.logger(TemplatePathsTraverser.class); public TemplatePathsTraverser(HugeGraph graph) { - super(graph, "template-paths"); + super(graph); } public Set templatePaths(Iterator sources, @@ -74,27 +74,28 @@ public Set templatePaths(Iterator sources, for (RepeatEdgeStep step : steps) { totalSteps += step.maxTimes; } - TraverseStrategy strategy = this.traverseStrategy( - totalSteps >= this.concurrentDepth()); - Traverser traverser = new Traverser(sourceList, targetList, steps, - withRing, capacity, limit, - strategy); + TraverseStrategy strategy = TraverseStrategy.create( + totalSteps >= this.concurrentDepth(), + this.graph()); + Traverser traverser = new Traverser(this, strategy, + sourceList, targetList, steps, + withRing, capacity, limit); do { // Forward traverser.forward(); - if (traverser.finish()) { + if (traverser.finished()) { return traverser.paths(); } // Backward traverser.backward(); - if (traverser.finish()) { + if (traverser.finished()) { return traverser.paths(); } } while (true); } - private class Traverser extends PathTraverser { + private static class Traverser extends PathTraverser { protected final List steps; @@ -106,10 +107,11 @@ private class Traverser extends PathTraverser { protected boolean sourceFinishOneStep = false; protected boolean targetFinishOneStep = false; - public Traverser(Collection sources, Collection targets, + public Traverser(HugeTraverser traverser, TraverseStrategy strategy, + Collection sources, Collection targets, List steps, boolean withRing, - long capacity, long limit, TraverseStrategy strategy) { - super(sources, targets, capacity, limit, strategy); + long capacity, long limit) { + super(traverser, strategy, sources, targets, capacity, limit); this.steps = steps; this.withRing = withRing; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java index 62429b6ee3..cbf06f7d78 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java @@ -19,16 +19,13 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; -import java.util.function.BiConsumer; import java.util.function.Consumer; import org.apache.commons.lang3.tuple.Pair; @@ -41,12 +38,8 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.config.CoreOptions; -import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.iterator.FilterIterator; import com.baidu.hugegraph.structure.HugeEdge; -import com.baidu.hugegraph.traversal.algorithm.strategy.ConcurrentTraverseStrategy; -import com.baidu.hugegraph.traversal.algorithm.strategy.SingleTraverseStrategy; -import com.baidu.hugegraph.traversal.algorithm.strategy.TraverseStrategy; import com.baidu.hugegraph.util.Consumers; import jersey.repackaged.com.google.common.base.Objects; @@ -54,35 +47,25 @@ public abstract class TpTraverser extends HugeTraverser implements AutoCloseable { - protected static ExecutorService executor; + private static final String EXECUTOR_NAME = "oltp"; + private static ExecutorService executor; protected TpTraverser(HugeGraph graph) { super(graph); - } - - protected TpTraverser(HugeGraph graph, String name) { - super(graph); if (executor == null) { int workers = this.config().get(CoreOptions.OLTP_CONCURRENT_THREADS); if (workers > 0) { - executor = Consumers.newThreadPool(name, workers); + executor = Consumers.newThreadPool(EXECUTOR_NAME, workers); } } } - protected int concurrentDepth() { - if (executor == null) { - return Integer.MAX_VALUE; - } - return this.config().get(CoreOptions.OLTP_CONCURRENT_DEPTH); - } - - protected HugeConfig config() { - return ((HugeConfig) this.graph().hugegraph().configuration()); - } - @Override public void close() { + // pass + } + + public static void destroy() { if (executor != null) { executor.shutdown(); executor = null; @@ -177,12 +160,6 @@ protected boolean match(Element elem, String key, Object value) { return p.isPresent() && Objects.equal(p.value(), value); } - protected TraverseStrategy traverseStrategy(boolean concurrent) { - return concurrent ? new ConcurrentTraverseStrategy(this.graph()) : - new SingleTraverseStrategy(this.graph()); - - } - public class ConcurrentMultiValuedMap extends ConcurrentHashMap> { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java index fa48fb5c80..9d20259516 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java @@ -19,15 +19,12 @@ package com.baidu.hugegraph.traversal.algorithm.strategy; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiConsumer; -import org.apache.tinkerpop.gremlin.structure.Edge; - import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; @@ -74,9 +71,4 @@ public void addNewVerticesToAll(Map> newVertices, vertices.addAll(entry.getKey(), entry.getValue()); } } - - @Override - public Iterator edgesOfVertex(Id source, EdgeStep edgeStep) { - return super.edgesOfVertex(source, edgeStep); - } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java index 44b5bdae84..23f3667d40 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm.strategy; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -27,8 +26,6 @@ import javax.ws.rs.core.MultivaluedMap; -import org.apache.tinkerpop.gremlin.structure.Edge; - import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; @@ -70,14 +67,9 @@ public void addNode(Map> vertices, Id id, Node node) { public void addNewVerticesToAll(Map> newVertices, Map> targets) { MultivaluedMap vertices = - (MultivaluedMap) targets; + (MultivaluedMap) targets; for (Map.Entry> entry : newVertices.entrySet()) { vertices.addAll(entry.getKey(), entry.getValue()); } } - - @Override - public Iterator edgesOfVertex(Id source, EdgeStep edgeStep) { - return super.edgesOfVertex(source, edgeStep); - } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java index 93a73b3068..1ab5a8de31 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java @@ -19,14 +19,12 @@ package com.baidu.hugegraph.traversal.algorithm.strategy; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; -import org.apache.tinkerpop.gremlin.structure.Edge; - +import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; @@ -48,5 +46,9 @@ public abstract void addNewVerticesToAll( Map> newVertices, Map> targets); - public Iterator edgesOfVertex(Id source, EdgeStep edgeStep); + public static TraverseStrategy create(boolean concurrent, HugeGraph graph) { + return concurrent ? new ConcurrentTraverseStrategy(graph) : + new SingleTraverseStrategy(graph); + + } } From 20c1ff5252049df1ab2f4ad36d6e18af70c71690 Mon Sep 17 00:00:00 2001 From: zhangyi51 Date: Wed, 21 Oct 2020 16:26:56 +0800 Subject: [PATCH 20/20] improve Change-Id: I882e472d001c40c0f1ffdc89c1c3c28bd6d34a17 --- .../main/java/com/baidu/hugegraph/HugeFactory.java | 4 ++-- .../algorithm/CollectionPathsTraverser.java | 1 - .../algorithm/JaccardSimilarTraverser.java | 2 +- .../traversal/algorithm/KneighborTraverser.java | 2 +- .../traversal/algorithm/KoutTraverser.java | 2 +- .../algorithm/MultiNodeShortestPathTraverser.java | 2 +- .../{TpTraverser.java => OltpTraverser.java} | 14 ++++++++++---- .../strategy/ConcurrentTraverseStrategy.java | 6 +++--- .../algorithm/strategy/SingleTraverseStrategy.java | 4 ++-- .../algorithm/strategy/TraverseStrategy.java | 1 - 10 files changed, 21 insertions(+), 17 deletions(-) rename hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/{TpTraverser.java => OltpTraverser.java} (94%) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java index c2b6f0c7a2..2e43f40c1b 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeFactory.java @@ -34,7 +34,7 @@ import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.event.EventHub; import com.baidu.hugegraph.task.TaskManager; -import com.baidu.hugegraph.traversal.algorithm.TpTraverser; +import com.baidu.hugegraph.traversal.algorithm.OltpTraverser; import com.baidu.hugegraph.type.define.SerialEnum; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; @@ -128,7 +128,7 @@ public static void shutdown(long timeout) { throw new TimeoutException(timeout + "s"); } TaskManager.instance().shutdown(timeout); - TpTraverser.destroy(); + OltpTraverser.destroy(); } catch (Throwable e) { LOG.error("Error while shutdown", e); throw new HugeException("Failed to shutdown", e); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java index 7da8ea0e51..6f1989464b 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CollectionPathsTraverser.java @@ -69,7 +69,6 @@ public Collection paths(Iterator sources, "but got: %s", MAX_VERTICES, sourceList.size()); checkPositive(depth, "max depth"); - TraverseStrategy strategy = TraverseStrategy.create( depth >= this.concurrentDepth(), this.graph()); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java index 70dd1b6a4f..82e679c250 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/JaccardSimilarTraverser.java @@ -36,7 +36,7 @@ import com.baidu.hugegraph.util.E; import com.google.common.collect.ImmutableMap; -public class JaccardSimilarTraverser extends TpTraverser { +public class JaccardSimilarTraverser extends OltpTraverser { public JaccardSimilarTraverser(HugeGraph graph) { super(graph); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java index 7ab46cbd47..3c68021b5e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KneighborTraverser.java @@ -27,7 +27,7 @@ import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; -public class KneighborTraverser extends TpTraverser { +public class KneighborTraverser extends OltpTraverser { public KneighborTraverser(HugeGraph graph) { super(graph); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java index c225065d17..fcfddf6bbc 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/KoutTraverser.java @@ -28,7 +28,7 @@ import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; -public class KoutTraverser extends TpTraverser { +public class KoutTraverser extends OltpTraverser { public KoutTraverser(HugeGraph graph) { super(graph); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java index 26d5fab2b9..633b5bd79f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/MultiNodeShortestPathTraverser.java @@ -37,7 +37,7 @@ import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.Path.EMPTY_PATH; -public class MultiNodeShortestPathTraverser extends TpTraverser { +public class MultiNodeShortestPathTraverser extends OltpTraverser { public MultiNodeShortestPathTraverser(HugeGraph graph) { super(graph); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/OltpTraverser.java similarity index 94% rename from hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java rename to hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/OltpTraverser.java index cbf06f7d78..b35c1713d2 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/TpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/OltpTraverser.java @@ -44,15 +44,21 @@ import jersey.repackaged.com.google.common.base.Objects; -public abstract class TpTraverser extends HugeTraverser - implements AutoCloseable { +public abstract class OltpTraverser extends HugeTraverser + implements AutoCloseable { private static final String EXECUTOR_NAME = "oltp"; private static ExecutorService executor; - protected TpTraverser(HugeGraph graph) { + protected OltpTraverser(HugeGraph graph) { super(graph); - if (executor == null) { + if (executor != null) { + return; + } + synchronized (OltpTraverser.class) { + if (executor != null) { + return; + } int workers = this.config().get(CoreOptions.OLTP_CONCURRENT_THREADS); if (workers > 0) { executor = Consumers.newThreadPool(EXECUTOR_NAME, workers); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java index 9d20259516..f856e0a47c 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/ConcurrentTraverseStrategy.java @@ -28,9 +28,9 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; -import com.baidu.hugegraph.traversal.algorithm.TpTraverser; +import com.baidu.hugegraph.traversal.algorithm.OltpTraverser; -public class ConcurrentTraverseStrategy extends TpTraverser +public class ConcurrentTraverseStrategy extends OltpTraverser implements TraverseStrategy { public ConcurrentTraverseStrategy(HugeGraph graph) { @@ -39,7 +39,7 @@ public ConcurrentTraverseStrategy(HugeGraph graph) { @Override public Map> newMultiValueMap() { - return new TpTraverser.ConcurrentMultiValuedMap<>(); + return new OltpTraverser.ConcurrentMultiValuedMap<>(); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java index 23f3667d40..822949d185 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/SingleTraverseStrategy.java @@ -30,9 +30,9 @@ import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.traversal.algorithm.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; -import com.baidu.hugegraph.traversal.algorithm.TpTraverser; +import com.baidu.hugegraph.traversal.algorithm.OltpTraverser; -public class SingleTraverseStrategy extends TpTraverser +public class SingleTraverseStrategy extends OltpTraverser implements TraverseStrategy { public SingleTraverseStrategy(HugeGraph graph) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java index 1ab5a8de31..cb81ad87f8 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/strategy/TraverseStrategy.java @@ -49,6 +49,5 @@ public abstract void addNewVerticesToAll( public static TraverseStrategy create(boolean concurrent, HugeGraph graph) { return concurrent ? new ConcurrentTraverseStrategy(graph) : new SingleTraverseStrategy(graph); - } }