From 23e3c31a5445cd472e16c6a625a119ca0ab4b04d Mon Sep 17 00:00:00 2001 From: zhoney Date: Wed, 9 Jun 2021 17:36:47 +0800 Subject: [PATCH] Collection optimize for OLTP algorithm and HugeElement (#1409) * add Eclipse Collection dependency * use primitive map for properties of HugeElement * from Map> to MutableIntObjectMap> * allow config oltp collections(JCF or EC) * add fastutil choice for collection * change Set of HugeVertex to List * use IntObjectMap and IntIntMap to improve shortest path * improve paths api by remove redundant sourcesAll and targetsAll * improve paths api performance by using IntObjectHashMap * extract shortestpath and paths record to hold traversed paths * implement Int2IntsMap for array paths record * extract Record and Records for paths and shortest path * adapt primitive collection for kout/kneighbor * add objectintmapping unit test * add UT for CollectionFactory and IdSet Change-Id: I501a18a70054fad3c40bd6b9e852a82a5c185949 --- .../api/traversers/AllShortestPathsAPI.java | 4 +- .../api/traversers/KneighborAPI.java | 26 +- .../hugegraph/api/traversers/KoutAPI.java | 27 +- .../hugegraph/api/traversers/PathsAPI.java | 10 +- .../api/traversers/PersonalRankAPI.java | 10 +- .../api/traversers/ShortestPathAPI.java | 5 +- .../api/traversers/TraverserAPI.java | 4 +- .../hugegraph/auth/HugeGraphAuthProxy.java | 3 +- hugegraph-core/pom.xml | 17 + .../java/com/baidu/hugegraph/HugeGraph.java | 3 +- .../baidu/hugegraph/StandardHugeGraph.java | 12 +- .../backend/cache/CachedGraphTransaction.java | 2 + .../cache/CachedSchemaTransaction.java | 2 + .../baidu/hugegraph/backend/id/EdgeId.java | 2 + .../backend/query/ConditionQuery.java | 2 + .../backend/serializer/BinarySerializer.java | 3 +- .../backend/store/ram/IntLongMap.java | 2 + .../hugegraph/backend/store/ram/RamTable.java | 4 + .../backend/tx/GraphTransaction.java | 2 + .../baidu/hugegraph/config/CoreOptions.java | 11 + .../baidu/hugegraph/structure/HugeEdge.java | 2 + .../hugegraph/structure/HugeElement.java | 59 ++- .../baidu/hugegraph/structure/HugeVertex.java | 50 ++- .../algorithm/CollectionPathsTraverser.java | 5 +- .../traversal/algorithm/CountTraverser.java | 3 +- .../algorithm/CustomizePathsTraverser.java | 9 +- .../CustomizedCrosspointsTraverser.java | 16 +- .../FusiformSimilarityTraverser.java | 40 +- .../traversal/algorithm/HugeTraverser.java | 178 +++++--- .../algorithm/JaccardSimilarTraverser.java | 8 +- .../algorithm/KneighborTraverser.java | 61 +-- .../traversal/algorithm/KoutTraverser.java | 95 ++--- .../MultiNodeShortestPathTraverser.java | 7 +- .../algorithm/NeighborRankTraverser.java | 20 +- .../traversal/algorithm/OltpTraverser.java | 56 +-- .../traversal/algorithm/PathsTraverser.java | 118 ++---- .../algorithm/PersonalRankTraverser.java | 14 +- .../algorithm/ShortestPathTraverser.java | 142 +++---- .../SingleSourceShortestPathTraverser.java | 9 +- .../algorithm/SubGraphTraverser.java | 5 +- .../algorithm/TemplatePathsTraverser.java | 5 +- .../algorithm/records/AbstractRecords.java | 64 +++ .../records/DoubleWayMultiPathsRecords.java | 210 ++++++++++ .../algorithm/records/KneighborRecords.java | 73 ++++ .../algorithm/records/KoutRecords.java | 65 +++ .../algorithm/records/PathsRecords.java | 30 ++ .../traversal/algorithm/records/Records.java | 41 ++ .../records/ShortestPathRecords.java | 117 ++++++ .../records/SingleWayMultiPathsRecords.java | 176 ++++++++ .../records/record/Int2ArrayRecord.java | 56 +++ .../records/record/Int2IntRecord.java | 60 +++ .../records/record/Int2SetRecord.java | 61 +++ .../algorithm/records/record/IntIterator.java | 57 +++ .../algorithm/records/record/Record.java | 33 ++ .../records/record/RecordFactory.java | 49 +++ .../algorithm/records/record/RecordType.java | 65 +++ .../algorithm/records/record/SyncRecord.java | 80 ++++ .../hugegraph/type/define/CollectionType.java | 67 +++ .../util/collection/CollectionFactory.java | 264 ++++++++++++ .../ConcurrentObjectIntMapping.java | 46 +++ .../hugegraph/util/collection/IdSet.java | 115 ++++++ .../util/collection/Int2IntsMap.java | 188 +++++++++ .../util/collection/MappingFactory.java | 33 ++ .../util/collection/ObjectIntMapping.java | 29 ++ .../SingleThreadObjectIntMapping.java | 76 ++++ .../baidu/hugegraph/dist/HugeGraphServer.java | 2 +- .../baidu/hugegraph/example/ExampleUtil.java | 4 +- .../baidu/hugegraph/core/EdgeCoreTest.java | 5 +- .../baidu/hugegraph/unit/UnitTestSuite.java | 10 +- .../unit/cache/CacheManagerTest.java | 2 +- .../baidu/hugegraph/unit/core/IdSetTest.java | 328 +++++++++++++++ .../hugegraph/unit/core/Int2IntsMapTest.java | 177 ++++++++ .../unit/core/ObjectIntMappingTest.java | 143 +++++++ .../unit/util/CollectionFactoryTest.java | 387 ++++++++++++++++++ .../hugegraph/unit/util/JsonUtilTest.java | 15 +- 75 files changed, 3663 insertions(+), 518 deletions(-) create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/AbstractRecords.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/DoubleWayMultiPathsRecords.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KneighborRecords.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KoutRecords.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/PathsRecords.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/Records.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/ShortestPathRecords.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/SingleWayMultiPathsRecords.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2ArrayRecord.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2IntRecord.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2SetRecord.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/IntIterator.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Record.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordFactory.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordType.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/SyncRecord.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/CollectionType.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/CollectionFactory.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ConcurrentObjectIntMapping.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/IdSet.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/Int2IntsMap.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/MappingFactory.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ObjectIntMapping.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/SingleThreadObjectIntMapping.java create mode 100644 hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/IdSetTest.java create mode 100644 hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/Int2IntsMapTest.java create mode 100644 hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/ObjectIntMappingTest.java create mode 100644 hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/CollectionFactoryTest.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 21371d06cb..cfbde599ee 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,11 +19,11 @@ 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_MAX_DEGREE; +import java.util.List; + import javax.inject.Singleton; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; 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 c912936e9a..4ba7819b80 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 @@ -22,10 +22,8 @@ import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_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; @@ -50,6 +48,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.records.KneighborRecords; import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.traversal.algorithm.KneighborTraverser; @@ -125,32 +124,27 @@ public String post(@Context GraphManager manager, EdgeStep step = step(g, request.step); - Set results; + KneighborRecords results; try (KneighborTraverser traverser = new KneighborTraverser(g)) { results = traverser.customizedKneighbor(sourceId, step, request.maxDepth, request.limit); } - Set neighbors = new HashSet<>(); - for (HugeTraverser.Node node : results) { - neighbors.add(node.id()); - } + Set neighbors = results.ids(request.limit); - List paths = new ArrayList<>(); + HugeTraverser.PathSet paths = new HugeTraverser.PathSet(); if (request.withPath) { - for (HugeTraverser.Node node : results) { - paths.add(new HugeTraverser.Path(node.path())); - } + paths.addAll(results.paths(request.limit)); } 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()); + ids.addAll(results.ids(request.limit)); + if (request.withPath) { + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } } if (!ids.isEmpty()) { iter = g.vertices(ids.toArray()); 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 c91b2fbb57..b6aa8d537f 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 @@ -23,10 +23,8 @@ import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_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; @@ -51,9 +49,9 @@ import com.baidu.hugegraph.core.GraphManager; import com.baidu.hugegraph.server.RestServer; import com.baidu.hugegraph.structure.HugeVertex; +import com.baidu.hugegraph.traversal.algorithm.records.KoutRecords; import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; -import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.Node; import com.baidu.hugegraph.traversal.algorithm.KoutTraverser; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; @@ -133,7 +131,7 @@ public String post(@Context GraphManager manager, EdgeStep step = step(g, request.step); - Set results; + KoutRecords results; try (KoutTraverser traverser = new KoutTraverser(g)) { results = traverser.customizedKout(sourceId, step, request.maxDepth, @@ -142,25 +140,20 @@ public String post(@Context GraphManager manager, request.limit); } - Set neighbors = new HashSet<>(); - for (HugeTraverser.Node node : results) { - neighbors.add(node.id()); - } + Set neighbors = results.ids(request.limit); - List paths = new ArrayList<>(); + HugeTraverser.PathSet paths = new HugeTraverser.PathSet(); if (request.withPath) { - for (HugeTraverser.Node node : results) { - paths.add(new HugeTraverser.Path(node.path())); - } + paths.addAll(results.paths(request.limit)); } 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()); + ids.addAll(results.ids(request.limit)); + if (request.withPath) { + for (HugeTraverser.Path p : paths) { + ids.addAll(p.vertices()); + } } if (!ids.isEmpty()) { iter = g.vertices(ids.toArray()); 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 cd1096e29d..095463b7cb 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,15 +19,15 @@ 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_MAX_DEGREE; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_PATHS_LIMIT; + 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_MAX_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; @@ -50,9 +50,9 @@ 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.steps.EdgeStep; import com.baidu.hugegraph.traversal.algorithm.HugeTraverser; import com.baidu.hugegraph.traversal.algorithm.PathsTraverser; +import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PersonalRankAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PersonalRankAPI.java index 563d08ca5a..5ec50e382a 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PersonalRankAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/traversers/PersonalRankAPI.java @@ -19,6 +19,11 @@ package com.baidu.hugegraph.api.traversers; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_LIMIT; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_DEGREE; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_DEPTH; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.NO_LIMIT; + import java.util.Map; import javax.inject.Singleton; @@ -43,11 +48,6 @@ import com.codahale.metrics.annotation.Timed; import com.fasterxml.jackson.annotation.JsonProperty; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_DEGREE; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_LIMIT; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_DEPTH; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.NO_LIMIT; - @Path("graphs/{graph}/traversers/personalrank") @Singleton public class PersonalRankAPI extends API { 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 189a770e64..87e2f7cd60 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,11 +19,11 @@ 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_MAX_DEGREE; +import java.util.List; + import javax.inject.Singleton; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; @@ -76,7 +76,6 @@ public String get(@Context GraphManager manager, "max degree '{}', skipped maxDegree '{}' and capacity '{}'", graph, source, target, direction, edgeLabel, depth, maxDegree, skipDegree, capacity); - Id sourceId = VertexAPI.checkAndParseVertexId(source); Id targetId = VertexAPI.checkAndParseVertexId(target); Directions dir = Directions.convert(EdgeAPI.parseDirection(direction)); 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 cab71015af..62c4cc46ac 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 @@ -19,6 +19,8 @@ package com.baidu.hugegraph.api.traversers; +import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_DEGREE; + import java.util.List; import java.util.Map; @@ -29,8 +31,6 @@ import com.fasterxml.jackson.annotation.JsonAlias; import com.fasterxml.jackson.annotation.JsonProperty; -import static com.baidu.hugegraph.traversal.algorithm.HugeTraverser.DEFAULT_MAX_DEGREE; - public class TraverserAPI extends API { protected static EdgeStep step(HugeGraph graph, Step step) { diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java index 34fcc6d495..807d3c4f71 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java @@ -65,6 +65,7 @@ import com.baidu.hugegraph.backend.store.raft.RaftGroupManager; import com.baidu.hugegraph.config.ConfigOption; import com.baidu.hugegraph.config.HugeConfig; +import com.baidu.hugegraph.config.TypedOption; import com.baidu.hugegraph.exception.NotSupportException; import com.baidu.hugegraph.iterator.FilterIterator; import com.baidu.hugegraph.rpc.RpcServiceConfig4Client; @@ -552,7 +553,7 @@ public long now() { } @Override - public V option(ConfigOption option) { + public V option(TypedOption option) { this.verifyAnyPermission(); return this.hugegraph.option(option); } diff --git a/hugegraph-core/pom.xml b/hugegraph-core/pom.xml index d9fb1a1700..0b70dbea3c 100644 --- a/hugegraph-core/pom.xml +++ b/hugegraph-core/pom.xml @@ -157,6 +157,23 @@ commons-compress 1.20 + + org.eclipse.collections + eclipse-collections-api + 10.4.0 + + + org.eclipse.collections + eclipse-collections + 10.4.0 + + + + + it.unimi.dsi + fastutil + 8.1.0 + diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java index ed26bbb5d0..e9b93f4d86 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java @@ -38,6 +38,7 @@ import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; import com.baidu.hugegraph.backend.store.raft.RaftGroupManager; import com.baidu.hugegraph.config.ConfigOption; +import com.baidu.hugegraph.config.TypedOption; import com.baidu.hugegraph.rpc.RpcServiceConfig4Client; import com.baidu.hugegraph.rpc.RpcServiceConfig4Server; import com.baidu.hugegraph.schema.EdgeLabel; @@ -170,7 +171,7 @@ public interface HugeGraph extends Graph { public long now(); - public V option(ConfigOption option); + public V option(TypedOption option); public void registerRpcServices(RpcServiceConfig4Server serverConfig, RpcServiceConfig4Client clientConfig); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java index cfa1aec501..08a2f998df 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -64,13 +64,14 @@ import com.baidu.hugegraph.backend.store.ram.RamTable; import com.baidu.hugegraph.backend.tx.GraphTransaction; import com.baidu.hugegraph.backend.tx.SchemaTransaction; -import com.baidu.hugegraph.config.ConfigOption; import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.config.HugeConfig; +import com.baidu.hugegraph.config.TypedOption; import com.baidu.hugegraph.event.EventHub; import com.baidu.hugegraph.event.EventListener; import com.baidu.hugegraph.exception.NotAllowException; import com.baidu.hugegraph.io.HugeGraphIoRegistry; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.rpc.RpcServiceConfig4Client; import com.baidu.hugegraph.rpc.RpcServiceConfig4Server; import com.baidu.hugegraph.schema.EdgeLabel; @@ -115,7 +116,7 @@ public class StandardHugeGraph implements HugeGraph { StandardHugeGraph.SysTransaction.class }; - public static final Set> ALLOWED_CONFIGS = ImmutableSet.of( + public static final Set> ALLOWED_CONFIGS = ImmutableSet.of( CoreOptions.TASK_WAIT_TIMEOUT, CoreOptions.TASK_SYNC_DELETION, CoreOptions.TASK_TTL_DELETE_BATCH, @@ -123,6 +124,7 @@ public class StandardHugeGraph implements HugeGraph { CoreOptions.TASK_RESULT_SIZE_LIMIT, CoreOptions.OLTP_CONCURRENT_THREADS, CoreOptions.OLTP_CONCURRENT_DEPTH, + CoreOptions.OLTP_COLLECTION_TYPE, CoreOptions.VERTEX_DEFAULT_LABEL, CoreOptions.VERTEX_ENCODE_PK_NUMBER ); @@ -453,6 +455,7 @@ private BackendStore loadSystemStore() { return this.storeProvider.loadSystemStore(name); } + @Watched private SchemaTransaction schemaTransaction() { this.checkGraphNotClosed(); /* @@ -472,6 +475,7 @@ private SysTransaction systemTransaction() { return this.tx.systemTransaction(); } + @Watched private GraphTransaction graphTransaction() { this.checkGraphNotClosed(); /* @@ -656,6 +660,7 @@ public Iterator edges(Object... objects) { } @Override + @Watched public Iterator edges(Query query) { return this.graphTransaction().queryEdges(query); } @@ -727,6 +732,7 @@ public Collection vertexLabels() { } @Override + @Watched public VertexLabel vertexLabelOrNone(Id id) { VertexLabel vl = this.schemaTransaction().getVertexLabel(id); if (vl == null) { @@ -972,7 +978,7 @@ public long now() { } @Override - public V option(ConfigOption option) { + public V option(TypedOption option) { HugeConfig config = this.configuration(); if (!ALLOWED_CONFIGS.contains(option)) { throw new NotAllowException("Not allowed to access config: %s", diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedGraphTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedGraphTransaction.java index a24d54df3d..37d7bf8cf6 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedGraphTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedGraphTransaction.java @@ -44,6 +44,7 @@ import com.baidu.hugegraph.exception.NotSupportException; import com.baidu.hugegraph.iterator.ExtendableIterator; import com.baidu.hugegraph.iterator.ListIterator; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.schema.IndexLabel; import com.baidu.hugegraph.structure.HugeEdge; import com.baidu.hugegraph.structure.HugeVertex; @@ -264,6 +265,7 @@ private Iterator queryVerticesByIds(IdQuery query) { } @Override + @Watched protected final Iterator queryEdgesFromBackend(Query query) { RamTable ramtable = this.params().ramtable(); if (ramtable != null && ramtable.matched(query)) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedSchemaTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedSchemaTransaction.java index 7adc1bf77a..f94d52d78d 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedSchemaTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedSchemaTransaction.java @@ -33,6 +33,7 @@ import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.event.EventHub; import com.baidu.hugegraph.event.EventListener; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.schema.SchemaElement; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.util.E; @@ -348,6 +349,7 @@ public void updateIfNeeded(V schema) { } } + @Watched public V get(HugeType type, Id id) { assert id.number(); long longId = id.asLong(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/id/EdgeId.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/id/EdgeId.java index 1c69e55cc4..60a5aa7194 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/id/EdgeId.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/id/EdgeId.java @@ -20,6 +20,7 @@ package com.baidu.hugegraph.backend.id; import com.baidu.hugegraph.exception.NotFoundException; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.structure.HugeVertex; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.Directions; @@ -76,6 +77,7 @@ public EdgeId(Id ownerVertexId, Directions direction, Id edgeLabelId, this.cache = null; } + @Watched public EdgeId switchDirection() { Directions direction = this.direction.opposite(); return new EdgeId(this.otherVertexId, direction, this.edgeLabelId, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java index 80e2bf9c18..d083b8f674 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java @@ -35,6 +35,7 @@ import com.baidu.hugegraph.backend.id.SplicingIdGenerator; import com.baidu.hugegraph.backend.query.Condition.Relation; import com.baidu.hugegraph.backend.query.Condition.RelationType; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.structure.HugeElement; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.HugeKeys; @@ -141,6 +142,7 @@ public List relations() { return relations; } + @Watched public T condition(Object key) { List values = new ArrayList<>(); for (Condition c : this.conditions) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java index e585135252..9e3441f67e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java @@ -22,7 +22,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.commons.lang.NotImplementedException; @@ -459,7 +458,7 @@ public BackendEntry writeEdgeProperty(HugeEdgeProperty prop) { @Override public HugeEdge readEdge(HugeGraph graph, BackendEntry bytesEntry) { HugeVertex vertex = this.readVertex(graph, bytesEntry); - Set edges = vertex.getEdges(); + Collection edges = vertex.getEdges(); E.checkState(edges.size() == 1, "Expect one edge in vertex, but got %s", edges.size()); return edges.iterator().next(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/IntLongMap.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/IntLongMap.java index 7bbb7b98c9..36ef9428fa 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/IntLongMap.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/IntLongMap.java @@ -64,6 +64,7 @@ public long get(int key) { @Override public void clear() { Arrays.fill(this.array, 0L); + this.size = 0; } @Override @@ -90,5 +91,6 @@ public void readFrom(DataInputStream buffer) throws IOException { long value = buffer.readLong(); this.array[i] = value; } + this.size = size; } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java index 01614a007d..728b995578 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java @@ -50,6 +50,7 @@ import com.baidu.hugegraph.backend.query.Condition; import com.baidu.hugegraph.backend.query.ConditionQuery; import com.baidu.hugegraph.backend.query.Query; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.schema.EdgeLabel; import com.baidu.hugegraph.schema.VertexLabel; import com.baidu.hugegraph.structure.HugeEdge; @@ -254,6 +255,7 @@ public long edgesSize() { return this.edges.size() - 1L; } + @Watched public boolean matched(Query query) { if (this.edgesSize() == 0L || this.loading) { return false; @@ -293,6 +295,7 @@ public boolean matched(Query query) { return matchedConds == cq.conditions().size(); } + @Watched public Iterator query(Query query) { assert this.matched(query); assert this.edgesSize() > 0; @@ -311,6 +314,7 @@ public Iterator query(Query query) { return this.query(owner.asLong(), dir, (int) label.asLong()); } + @Watched public Iterator query(long owner, Directions dir, int label) { if (this.loading) { // don't query when loading diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java index a2af8b0d3d..702bb1bb59 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java @@ -896,6 +896,7 @@ public Iterator queryEdges() { return this.queryEdges(q); } + @Watched public Iterator queryEdges(Query query) { E.checkArgument(this.removedEdges.isEmpty() || query.noLimit(), "It's not allowed to query with limit when " + @@ -1131,6 +1132,7 @@ public void removeEdgeProperty(HugeEdgeProperty prop) { * @param edgeLabels edge labels of queried edges * @return constructed condition query */ + @Watched public static ConditionQuery constructEdgesQuery(Id sourceVertex, Directions direction, Id... edgeLabels) { 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 24fcf606ae..ddff4e6019 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 @@ -26,6 +26,7 @@ import static com.baidu.hugegraph.config.OptionChecker.rangeInt; import com.baidu.hugegraph.backend.query.Query; +import com.baidu.hugegraph.type.define.CollectionType; import com.baidu.hugegraph.util.Bytes; public class CoreOptions extends OptionHolder { @@ -623,4 +624,14 @@ public static synchronized CoreOptions instance() { rangeInt(0, 65535), 10 ); + + public static final ConfigConvOption OLTP_COLLECTION_TYPE = + new ConfigConvOption<>( + "oltp.collection_type", + "The implementation type of collections " + + "used in oltp algorithm.", + allowValues("JCF", "EC", "FU"), + CollectionType::valueOf, + "EC" + ); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeEdge.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeEdge.java index 70118634c7..f579fc9b13 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeEdge.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeEdge.java @@ -370,6 +370,7 @@ public void vertices(boolean outEdge, HugeVertex owner, HugeVertex other) { } } + @Watched public HugeEdge switchOwner() { HugeEdge edge = this.clone(); edge.isOutEdge = !edge.isOutEdge; @@ -488,6 +489,7 @@ public static final EdgeId getIdValue(Object idValue, return EdgeId.parse(id.asString(), returnNullIfError); } + @Watched public static HugeEdge constructEdge(HugeVertex ownerVertex, boolean isOutEdge, EdgeLabel edgeLabel, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java index 517fc38679..825eb73585 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -35,6 +34,9 @@ import org.apache.tinkerpop.gremlin.structure.Property; import org.apache.tinkerpop.gremlin.structure.T; import org.apache.tinkerpop.gremlin.structure.util.ElementHelper; +import org.eclipse.collections.api.map.primitive.MutableIntObjectMap; +import org.eclipse.collections.api.tuple.primitive.IntObjectPair; +import org.eclipse.collections.impl.map.mutable.primitive.IntObjectHashMap; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.EdgeId; @@ -52,15 +54,16 @@ import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; -import com.google.common.collect.ImmutableMap; +import com.baidu.hugegraph.util.collection.CollectionFactory; public abstract class HugeElement implements Element, GraphType, Idfiable { - private static final Map> EMPTY_MAP = ImmutableMap.of(); + private static final MutableIntObjectMap> EMPTY_MAP = + CollectionFactory.newIntObjectMap(); private static final int MAX_PROPERTIES = BytesBuffer.UINT16_MAX; private final HugeGraph graph; - private Map> properties; + private MutableIntObjectMap> properties; private long expiredTime; // TODO: move into properties to keep small object @@ -98,7 +101,7 @@ protected void updateToDefaultValueIfNone() { this.defaultValueUpdated = true; // Set default value if needed for (Id pkeyId : this.schemaLabel().properties()) { - if (this.properties.containsKey(pkeyId)) { + if (this.properties.containsKey(intFromId(pkeyId))) { continue; } PropertyKey pkey = this.graph().propertyKey(pkeyId); @@ -198,28 +201,36 @@ public boolean hasTtl() { return this.schemaLabel().ttl() > 0L; } + // TODO: return MutableIntObjectMap> public Map> getProperties() { - return Collections.unmodifiableMap(this.properties); + Map> props = new HashMap<>(); + for (IntObjectPair> e : this.properties.keyValuesView()) { + props.put(IdGenerator.of(e.getOne()), e.getTwo()); + } + return props; } + // TODO: return MutableIntObjectMap> public Map> getFilledProperties() { this.ensureFilledProperties(true); return this.getProperties(); } + // TODO: return MutableIntObjectMap> public Map getPropertiesMap() { Map props = new HashMap<>(); - for (Map.Entry> e : this.properties.entrySet()) { - props.put(e.getKey(), e.getValue().value()); + for (IntObjectPair> e : this.properties.keyValuesView()) { + props.put(IdGenerator.of(e.getOne()), e.getTwo().value()); } return props; } + // TODO: return MutableIntObjectMap> public Map> getAggregateProperties() { Map> aggrProps = new HashMap<>(); - for (Map.Entry> e : this.properties.entrySet()) { - if (e.getValue().type().isAggregateProperty()) { - aggrProps.put(e.getKey(), e.getValue()); + for (IntObjectPair> e : this.properties.keyValuesView()) { + if (e.getTwo().type().isAggregateProperty()) { + aggrProps.put(IdGenerator.of(e.getOne()), e.getTwo()); } } return aggrProps; @@ -227,12 +238,12 @@ public Map> getAggregateProperties() { @SuppressWarnings("unchecked") public HugeProperty getProperty(Id key) { - return (HugeProperty) this.properties.get(key); + return (HugeProperty) this.properties.get(intFromId(key)); } @SuppressWarnings("unchecked") public V getPropertyValue(Id key) { - HugeProperty prop = this.properties.get(key); + HugeProperty prop = this.properties.get(intFromId(key)); if (prop == null) { return null; } @@ -240,7 +251,7 @@ public V getPropertyValue(Id key) { } public boolean hasProperty(Id key) { - return this.properties.containsKey(key); + return this.properties.containsKey(intFromId(key)); } public boolean hasProperties() { @@ -266,17 +277,18 @@ public int sizeOfSubProperties() { @Watched(prefix = "element") public HugeProperty setProperty(HugeProperty prop) { if (this.properties == EMPTY_MAP) { - this.properties = new HashMap<>(); + this.properties = CollectionFactory.newIntObjectMap(); } PropertyKey pkey = prop.propertyKey(); - E.checkArgument(this.properties.containsKey(pkey.id()) || + + E.checkArgument(this.properties.containsKey(intFromId(pkey.id())) || this.properties.size() < MAX_PROPERTIES, "Exceeded the maximum number of properties"); - return this.properties.put(pkey.id(), prop); + return this.properties.put(intFromId(pkey.id()), prop); } public HugeProperty removeProperty(Id key) { - return this.properties.remove(key); + return this.properties.remove(intFromId(key)); } public HugeProperty addProperty(PropertyKey pkey, V value) { @@ -353,7 +365,7 @@ private HugeProperty addProperty(PropertyKey pkey, V value, } public void resetProperties() { - this.properties = new HashMap<>(); + this.properties = CollectionFactory.newIntObjectMap(); this.propLoaded = false; } @@ -361,7 +373,8 @@ protected void copyProperties(HugeElement element) { if (element.properties == EMPTY_MAP) { this.properties = EMPTY_MAP; } else { - this.properties = new HashMap<>(element.properties); + this.properties = CollectionFactory.newIntObjectMap( + element.properties); } this.propLoaded = true; } @@ -493,6 +506,12 @@ public static final Object getLabelValue(Object... keyValues) { return labelValue; } + public static int intFromId(Id id) { + E.checkArgument(id instanceof IdGenerator.LongId, + "Can't get number from %s(%s)", id, id.getClass()); + return ((IdGenerator.LongId) id).intValue(); + } + public static final class ElementKeys { private Object label = null; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeVertex.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeVertex.java index 7d1b0246fb..98e24143d3 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeVertex.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeVertex.java @@ -53,20 +53,21 @@ import com.baidu.hugegraph.schema.VertexLabel; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.Cardinality; +import com.baidu.hugegraph.type.define.CollectionType; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.type.define.IdStrategy; import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.InsertionOrderUtil; -import com.google.common.collect.ImmutableSet; +import com.baidu.hugegraph.util.collection.CollectionFactory; +import com.google.common.collect.ImmutableList; public class HugeVertex extends HugeElement implements Vertex, Cloneable { - private static final Set EMPTY_SET = ImmutableSet.of(); + private static final List EMPTY_LIST = ImmutableList.of(); private Id id; private VertexLabel label; - private Set edges; + protected Collection edges; public HugeVertex(final HugeGraph graph, Id id, VertexLabel label) { super(graph); @@ -75,7 +76,7 @@ public HugeVertex(final HugeGraph graph, Id id, VertexLabel label) { this.label = label; this.id = id; - this.edges = EMPTY_SET; + this.edges = EMPTY_LIST; if (this.id != null) { if (label.idStrategy() == IdStrategy.CUSTOMIZE_UUID) { this.assignId(id); @@ -217,12 +218,15 @@ public boolean existsEdges() { return this.edges.size() > 0; } - public Set getEdges() { - return Collections.unmodifiableSet(this.edges); + public Collection getEdges() { + return Collections.unmodifiableCollection(this.edges); } public void resetEdges() { - this.edges = InsertionOrderUtil.newSet(); + /* + * Use List to hold edges to reduce memory usage and operation time. + */ + this.edges = newList(); } public void removeEdge(HugeEdge edge) { @@ -230,8 +234,8 @@ public void removeEdge(HugeEdge edge) { } public void addEdge(HugeEdge edge) { - if (this.edges == EMPTY_SET) { - this.edges = InsertionOrderUtil.newSet(); + if (this.edges == EMPTY_LIST) { + this.edges = newList(); } this.edges.add(edge); } @@ -323,6 +327,7 @@ public HugeEdge constructEdge(String label, HugeVertex vertex, * Add edge with direction OUT * @param edge the out edge */ + @Watched public void addOutEdge(HugeEdge edge) { if (edge.ownerVertex() == null) { edge.sourceVertex(this); @@ -337,6 +342,7 @@ public void addOutEdge(HugeEdge edge) { * Add edge with direction IN * @param edge the in edge */ + @Watched public void addInEdge(HugeEdge edge) { if (edge.ownerVertex() == null) { edge.targetVertex(this); @@ -639,6 +645,14 @@ public static HugeVertex create(final GraphTransaction tx, return new HugeVertex4Insert(tx, id, label); } + private static Set newSet() { + return CollectionFactory.newSet(CollectionType.EC); + } + + private static List newList() { + return CollectionFactory.newList(CollectionType.EC); + } + private static final class HugeVertex4Insert extends HugeVertex { private GraphTransaction tx; @@ -646,10 +660,26 @@ private static final class HugeVertex4Insert extends HugeVertex { public HugeVertex4Insert(final GraphTransaction tx, Id id, VertexLabel label) { super(tx.graph(), id, label); + /* + * Use Set to hold edges inserted into vertex + * to avoid duplicated edges + */ + this.edges = newSet(); this.tx = tx; this.fresh(true); } + public void resetEdges() { + this.edges = newSet(); + } + + public void addEdge(HugeEdge edge) { + if (this.edges == EMPTY_LIST) { + this.edges = newSet(); + } + this.edges.add(edge); + } + @Override public void committed() { super.committed(); 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 da76202e5c..2562697398 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 @@ -19,7 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -48,7 +47,7 @@ public Collection paths(Iterator sources, checkCapacity(capacity); checkLimit(limit); - List sourceList = new ArrayList<>(); + List sourceList = newList(); while (sources.hasNext()) { sourceList.add(((HugeVertex) sources.next()).id()); } @@ -56,7 +55,7 @@ public Collection paths(Iterator sources, 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<>(); + List targetList = newList(); while (targets.hasNext()) { targetList.add(((HugeVertex) targets.next()).id()); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CountTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CountTraverser.java index ee27b40c44..af8ad2c830 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CountTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CountTraverser.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -40,7 +39,7 @@ public class CountTraverser extends HugeTraverser { private boolean containsTraversed = false; private long dedupSize = 1000000L; - private final Set dedupSet = new HashSet<>(); + private final Set dedupSet = newIdSet(); private final MutableLong count = new MutableLong(0L); public CountTraverser(HugeGraph graph) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizePathsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizePathsTraverser.java index 4c1f73eb09..ad5bf5c80b 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizePathsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizePathsTraverser.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -74,7 +73,7 @@ public List customizedPaths(Iterator vertices, // Traversal vertices of previous level for (Map.Entry> entry : sources.entrySet()) { - List adjacency = new ArrayList<>(); + List adjacency = newList(); edges = this.edgesOfVertex(entry.getKey(), step.step()); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); @@ -122,7 +121,7 @@ public List customizedPaths(Iterator vertices, if (stepNum != 0) { return ImmutableList.of(); } - List paths = new ArrayList<>(); + List paths = newList(); for (List nodes : newVertices.values()) { for (Node n : nodes) { if (sorted) { @@ -155,7 +154,7 @@ private static List sample(List nodes, long sample) { if (nodes.size() <= sample) { return nodes; } - List result = new ArrayList<>((int) sample); + List result = newList((int) sample); int size = nodes.size(); for (int random : CollectionUtil.randomSet(0, size, (int) sample)) { result.add(nodes.get(random)); @@ -173,7 +172,7 @@ public WeightNode(Id id, Node parent, double weight) { } public List weights() { - List weights = new ArrayList<>(); + List weights = newList(); WeightNode current = this; while (current.parent() != null) { weights.add(current.weight); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedCrosspointsTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedCrosspointsTraverser.java index 4345646d55..253c7bb200 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedCrosspointsTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/CustomizedCrosspointsTraverser.java @@ -19,9 +19,7 @@ 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; @@ -60,14 +58,14 @@ public CrosspointsPaths crosspointsPaths(Iterator vertices, checkCapacity(capacity); checkLimit(limit); MultivaluedMap initialSources = newMultivalueMap(); - List verticesList = new ArrayList<>(); + List verticesList = newList(); while (vertices.hasNext()) { HugeVertex vertex = (HugeVertex) vertices.next(); verticesList.add(vertex); Node node = new Node(vertex.id(), null); initialSources.add(vertex.id(), node); } - List paths = new ArrayList<>(); + List paths = newList(); for (PathPattern pathPattern : pathPatterns) { MultivaluedMap sources = initialSources; @@ -81,7 +79,7 @@ public CrosspointsPaths crosspointsPaths(Iterator vertices, // Traversal vertices of previous level for (Map.Entry> entry : sources.entrySet()) { - List adjacency = new ArrayList<>(); + List adjacency = newList(); edges = this.edgesOfVertex(entry.getKey(), step.edgeStep); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); @@ -151,11 +149,11 @@ private static CrosspointsPaths intersectionPaths(List sources, // Limit intersection number to limit crosspoints vertices in result int size = intersection.size(); if (limit != NO_LIMIT && size > limit) { - intersection = new ArrayList<>(intersection).subList(0, size - 1); + intersection = newList(intersection).subList(0, size - 1); } // Filter intersection paths - List results = new ArrayList<>(); + List results = newList(); for (Path path : paths) { List vertices = path.vertices(); int length = vertices.size(); @@ -163,7 +161,7 @@ private static CrosspointsPaths intersectionPaths(List sources, results.add(path); } } - return new CrosspointsPaths(new HashSet<>(intersection), results); + return new CrosspointsPaths(newSet(intersection), results); } public static class PathPattern { @@ -171,7 +169,7 @@ public static class PathPattern { private List steps; public PathPattern() { - this.steps = new ArrayList<>(); + this.steps = newList(); } public List steps() { 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 320833f1cd..39f807a21f 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 @@ -19,8 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -106,9 +104,9 @@ private Set fusiformSimilarityForVertex( // Get similar nodes and counts Iterator edges = this.edgesOfVertex(vertex.id(), direction, labelId, degree); - Map similars = new HashMap<>(); + Map similars = newMap(); MultivaluedMap intermediaries = new MultivaluedHashMap<>(); - Set neighbors = new HashSet<>(); + Set neighbors = newIdSet(); while (edges.hasNext()) { Id target = ((HugeEdge) edges.next()).id().otherVertexId(); if (neighbors.contains(target)) { @@ -120,7 +118,7 @@ private Set fusiformSimilarityForVertex( Directions backDir = direction.opposite(); Iterator backEdges = this.edgesOfVertex(target, backDir, labelId, degree); - Set currentSimilars = new HashSet<>(); + Set currentSimilars = newIdSet(); while (backEdges.hasNext()) { Id node = ((HugeEdge) backEdges.next()).id().otherVertexId(); if (currentSimilars.contains(node)) { @@ -149,7 +147,7 @@ private Set fusiformSimilarityForVertex( } // Match alpha double neighborNum = neighbors.size(); - Map matchedAlpha = new HashMap<>(); + Map matchedAlpha = newMap(); for (Map.Entry entry : similars.entrySet()) { double score = entry.getValue().intValue() / neighborNum; if (score >= alpha) { @@ -169,7 +167,7 @@ private Set fusiformSimilarityForVertex( } // Filter by groupCount by property if (groupProperty != null) { - Set values = new HashSet<>(); + Set values = newSet(); // Add groupProperty value of source vertex values.add(vertex.value(groupProperty)); for (Id id : topN.keySet()) { @@ -226,7 +224,7 @@ private boolean matchMinNeighborCount(HugeVertex vertex, neighborCount = IteratorUtils.count(edges); } else { edges = this.edgesOfVertex(vertex.id(), direction, labelId, degree); - Set neighbors = new HashSet<>(); + Set neighbors = newIdSet(); while (edges.hasNext()) { Id target = ((HugeEdge) edges.next()).id().otherVertexId(); neighbors.add(target); @@ -252,8 +250,8 @@ public static class Similar { public Similar(Id id, double score, List intermediaries) { this.id = id; this.score = score; - assert new HashSet<>(intermediaries).size() == - intermediaries.size() : "Invalid intermediaries"; + assert newSet(intermediaries).size() == intermediaries.size() : + "Invalid intermediaries"; this.intermediaries = intermediaries; } @@ -275,14 +273,20 @@ public Map toMap() { } } - public static class SimilarsMap extends HashMap> { + public static class SimilarsMap { private static final long serialVersionUID = -1906770930513268291L; + private Map> similars = newMap(); + + public void put(Id id, Set similars) { + this.similars.put(id, similars); + } + public Set vertices() { - Set vertices = new HashSet<>(); - vertices.addAll(this.keySet()); - for (Set similars : this.values()) { + Set vertices = newIdSet(); + vertices.addAll(this.similars.keySet()); + for (Set similars : this.similars.values()) { for (Similar similar : similars) { vertices.add(similar.id()); vertices.addAll(similar.intermediaries()); @@ -292,8 +296,8 @@ public Set vertices() { } public Map>> toMap() { - Map>> results = new HashMap<>(); - for (Map.Entry> entry : this.entrySet()) { + Map>> results = newMap(); + for (Map.Entry> entry : this.similars.entrySet()) { Id source = entry.getKey(); Set similars = entry.getValue(); Set> result = InsertionOrderUtil.newSet(); @@ -304,5 +308,9 @@ public Map>> toMap() { } return results; } + + public boolean isEmpty() { + return this.similars.isEmpty(); + } } } 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 3ec1362b8b..ada4854a4c 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 @@ -19,10 +19,8 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -50,16 +48,19 @@ import com.baidu.hugegraph.iterator.FilterIterator; import com.baidu.hugegraph.iterator.LimitIterator; import com.baidu.hugegraph.iterator.MapperIterator; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.schema.SchemaLabel; import com.baidu.hugegraph.structure.HugeEdge; import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.traversal.optimize.TraversalUtil; import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.type.define.CollectionType; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.InsertionOrderUtil; +import com.baidu.hugegraph.util.collection.CollectionFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -68,6 +69,8 @@ public class HugeTraverser { private HugeGraph graph; + private static CollectionFactory collectionFactory; + public static final String DEFAULT_CAPACITY = "10000000"; public static final String DEFAULT_ELEMENTS_LIMIT = "10000000"; public static final String DEFAULT_PATHS_LIMIT = "10"; @@ -87,6 +90,9 @@ public class HugeTraverser { public HugeTraverser(HugeGraph graph) { this.graph = graph; + if (collectionFactory == null) { + collectionFactory = new CollectionFactory(this.collectionType()); + } } public HugeGraph graph() { @@ -97,6 +103,10 @@ protected int concurrentDepth() { return this.graph.option(CoreOptions.OLTP_CONCURRENT_DEPTH); } + private CollectionType collectionType() { + return this.graph.option(CoreOptions.OLTP_COLLECTION_TYPE); + } + protected Set adjacentVertices(Id sourceV, Set vertices, Directions dir, Id label, Set excluded, long degree, @@ -105,7 +115,7 @@ protected Set adjacentVertices(Id sourceV, Set vertices, return ImmutableSet.of(); } - Set neighbors = newSet(); + Set neighbors = newIdSet(); for (Id source : vertices) { Iterator edges = this.edgesOfVertex(source, dir, label, degree); @@ -137,7 +147,7 @@ protected Iterator adjacentVertices(Id source, Directions dir, } protected Set adjacentVertices(Id source, EdgeStep step) { - Set neighbors = new HashSet<>(); + Set neighbors = newSet(); Iterator edges = this.edgesOfVertex(source, step); while (edges.hasNext()) { neighbors.add(((HugeEdge) edges.next()).id().otherVertexId()); @@ -145,30 +155,7 @@ protected Set adjacentVertices(Id source, EdgeStep step) { return neighbors; } - protected Set adjacentVertices(Id start, 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); - boolean matchExcluded = (excluded != null && - excluded.contains(kNode)); - if (matchExcluded || neighbors.contains(kNode) || - start.equals(kNode.id())) { - continue; - } - neighbors.add(kNode); - if (remaining != NO_LIMIT && --remaining <= 0L) { - return neighbors; - } - } - } - return neighbors; - } - + @Watched protected Iterator edgesOfVertex(Id source, Directions dir, Id label, long limit) { Id[] labels = {}; @@ -183,6 +170,7 @@ protected Iterator edgesOfVertex(Id source, Directions dir, return this.graph.edges(query); } + @Watched protected Iterator edgesOfVertex(Id source, Directions dir, Map labels, long limit) { if (labels == null || labels.isEmpty()) { @@ -413,7 +401,7 @@ public static Iterator skipSuperNodeIfNeeded(Iterator edges, if (skipDegree <= 0L) { return edges; } - List edgeList = new ArrayList<>(); + List edgeList = newList(); for (int i = 1; edges.hasNext(); i++) { Edge edge = edges.next(); if (i <= degree) { @@ -426,6 +414,10 @@ public static Iterator skipSuperNodeIfNeeded(Iterator edges, return edgeList.iterator(); } + protected static Set newIdSet() { + return collectionFactory.newIdSet(); + } + protected static Set newSet() { return newSet(false); } @@ -434,12 +426,36 @@ protected static Set newSet(boolean concurrent) { if (concurrent) { return ConcurrentHashMap.newKeySet(); } else { - return new HashSet<>(); + return collectionFactory.newSet(); } } + protected static Set newSet(int initialCapacity) { + return collectionFactory.newSet(initialCapacity); + } + + protected static Set newSet(Collection collection) { + return collectionFactory.newSet(collection); + } + + protected static List newList() { + return collectionFactory.newList(); + } + + protected static List newList(int initialCapacity) { + return collectionFactory.newList(initialCapacity); + } + + protected static List newList(Collection collection) { + return collectionFactory.newList(collection); + } + protected static Map newMap() { - return new HashMap<>(); + return collectionFactory.newMap(); + } + + protected static Map newMap(int initialCapacity) { + return collectionFactory.newMap(initialCapacity); } protected static MultivaluedMap newMultivalueMap() { @@ -490,7 +506,7 @@ public Node parent() { } public List path() { - List ids = new ArrayList<>(); + List ids = newList(); Node current = this; do { ids.add(current.id); @@ -536,22 +552,6 @@ public String toString() { } } - 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()); @@ -572,6 +572,10 @@ public Id crosspoint() { return this.crosspoint; } + public void addToLast(Id id) { + this.vertices.add(id); + } + public List vertices() { return this.vertices; } @@ -621,13 +625,83 @@ public boolean equals(Object other) { } } - public static class PathSet extends HashSet { + public static class PathSet implements Set { private static final long serialVersionUID = -8237531948776524872L; + private Set paths = newSet(); + + public boolean add(Path path) { + return this.paths.add(path); + } + + @Override + public boolean remove(Object o) { + return this.paths.remove(o); + } + + @Override + public boolean containsAll(Collection c) { + return this.paths.containsAll(c); + } + + @Override + public boolean addAll(Collection c) { + return this.paths.addAll(c); + } + + @Override + public boolean retainAll(Collection c) { + return this.paths.retainAll(c); + } + + @Override + public boolean removeAll(Collection c) { + return this.paths.removeAll(c); + } + + @Override + public void clear() { + this.paths.clear(); + } + + @Override + public boolean isEmpty() { + return this.paths.isEmpty(); + } + + @Override + public boolean contains(Object o) { + return this.paths.contains(o); + } + + @Override + public int size() { + return this.paths.size(); + } + + @Override + public Iterator iterator() { + return this.paths.iterator(); + } + + @Override + public Object[] toArray() { + return this.paths.toArray(); + } + + @Override + public T[] toArray(T[] a) { + return this.paths.toArray(a); + } + + public boolean addAll(PathSet paths) { + return this.paths.addAll(paths.paths); + } + public Set vertices() { - Set vertices = new HashSet<>(); - for (Path path : this) { + Set vertices = newIdSet(); + for (Path path : this.paths) { vertices.addAll(path.vertices()); } return vertices; 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 af9f321cb2..cc0571c6b4 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 @@ -19,8 +19,6 @@ 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; @@ -151,7 +149,7 @@ public Map jaccardSimilarsConcurrent(Id source, EdgeStep step, public Map jaccardSimilarsSingle(Id source, EdgeStep step, long capacity) { long count = 0L; - Set accessed = new HashSet<>(); + Set accessed = newIdSet(); accessed.add(source); reachCapacity(++count, capacity); @@ -163,9 +161,9 @@ public Map jaccardSimilarsSingle(Id source, EdgeStep step, return ImmutableMap.of(); } - Map results = new HashMap<>(); + Map results = newMap(); Set layer2s; - Set layer2All = new HashSet<>(); + Set layer2All = newIdSet(); double jaccardSimilarity; for (Id neighbor : layer1s) { // Skip if accessed already 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 3403a53c25..55c71c5c0a 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 @@ -21,9 +21,15 @@ import java.util.Iterator; import java.util.Set; +import java.util.function.Consumer; + +import org.apache.tinkerpop.gremlin.structure.Edge; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.traversal.algorithm.records.KneighborRecords; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; @@ -56,7 +62,7 @@ public Set kneighbor(Id sourceV, Directions dir, latest = this.adjacentVertices(sourceV, latest, dir, labelId, all, degree, remaining); all.addAll(latest); - if (limit != NO_LIMIT && all.size() >= limit) { + if (reachLimit(limit, all.size())) { break; } } @@ -64,45 +70,40 @@ public Set kneighbor(Id sourceV, Directions dir, return all; } - public Set customizedKneighbor(Id source, EdgeStep step, - int maxDepth, long limit) { + public KneighborRecords 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); - boolean single = maxDepth < this.concurrentDepth() || - step.direction() != Directions.BOTH; - return this.customizedKneighbor(source, step, maxDepth, - limit, single); - } - - public Set customizedKneighbor(Id source, EdgeStep step, int maxDepth, - long limit, boolean single) { - Set latest = newSet(single); - Set all = newSet(single); + boolean concurrent = maxDepth >= this.concurrentDepth() && + step.direction() == Directions.BOTH; - Node sourceV = new KNode(source, null); + KneighborRecords records = new KneighborRecords(RecordType.INT, + concurrent, + source, true); - latest.add(sourceV); + Consumer consumer = v -> { + if (this.reachLimit(limit, records.size())) { + return; + } + Iterator edges = edgesOfVertex(v, step); + while (!this.reachLimit(limit, records.size()) && edges.hasNext()) { + Id target = ((HugeEdge) edges.next()).id().otherVertexId(); + records.addPath(v, target); + } + }; while (maxDepth-- > 0) { - long remaining = limit == NO_LIMIT ? NO_LIMIT : limit - all.size(); - latest = this.adjacentVertices(source, latest, step, all, - remaining, single); - 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); - } + records.startOneLayer(true); + traverseIds(records.keys(), consumer, concurrent); + records.finishOneLayer(); } + return records; + } - return all; + private boolean reachLimit(long limit, int size) { + return limit != NO_LIMIT && size >= limit; } } 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 ec8e108c94..b434a102ad 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 @@ -19,14 +19,20 @@ package com.baidu.hugegraph.traversal.algorithm; +import java.util.Iterator; import java.util.Set; +import java.util.function.Consumer; + +import org.apache.tinkerpop.gremlin.structure.Edge; import com.baidu.hugegraph.HugeException; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.traversal.algorithm.records.KoutRecords; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.type.define.Directions; -import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; public class KoutTraverser extends OltpTraverser { @@ -55,10 +61,10 @@ public Set kout(Id sourceV, Directions dir, String label, Id labelId = this.getEdgeLabelId(label); - Set latest = newSet(); + Set latest = newIdSet(); latest.add(sourceV); - Set all = newSet(); + Set all = newIdSet(); all.add(sourceV); long remaining = capacity == NO_LIMIT ? @@ -92,67 +98,54 @@ public Set kout(Id sourceV, Directions dir, String label, return latest; } - public Set customizedKout(Id source, EdgeStep step, int maxDepth, - boolean nearest, long capacity, - long limit) { + public KoutRecords 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; - 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); - } - - return results; - } - - 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); - - 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(source, latest, step, all, - remaining, single); - all.addAll(latest); - } else { - latest = this.adjacentVertices(source, latest, step, null, - remaining, single); + long[] depth = new long[1]; + depth[0] = maxDepth; + boolean concurrent = maxDepth >= this.concurrentDepth() && + step.direction() == Directions.BOTH; + KoutRecords records = new KoutRecords(RecordType.INT, concurrent, + source, nearest); + + Consumer consumer = v -> { + if (this.reachLimit(limit, depth[0], records.size())) { + return; } - if (capacity != NO_LIMIT) { - // Update 'remaining' value to record remaining capacity - remaining -= latest.size(); - reachCapacity(remaining, capacity, depth); + Iterator edges = edgesOfVertex(v, step); + while (!this.reachLimit(limit, depth[0], records.size()) && + edges.hasNext()) { + Id target = ((HugeEdge) edges.next()).id().otherVertexId(); + records.addPath(v, target); + this.checkCapacity(capacity, records.accessed(), depth[0]); } - } + }; - return latest; + while (depth[0]-- > 0) { + records.startOneLayer(true); + traverseIds(records.keys(), consumer, concurrent); + records.finishOneLayer(); + } + return records; } - private static void reachCapacity(long remaining, long capacity, - int depth) { - if (remaining <= 0 && depth > 0) { + private void checkCapacity(long capacity, long accessed, long depth) { + if (capacity == NO_LIMIT) { + return; + } + if (accessed >= capacity && depth > 0) { throw new HugeException( "Reach capacity '%s' while remaining depth '%s'", capacity, depth); } } + + private boolean reachLimit(long limit, long depth, int size) { + return limit != NO_LIMIT && depth <= 0 && size >= limit; + } } 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 de047284c3..2db85dec20 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 @@ -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.concurrent.CopyOnWriteArrayList; @@ -51,7 +50,7 @@ public List multiNodeShortestPath(Iterator 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<>(); + List> pairs = newList(); cmn(vertexList, vertexCount, 2, 0, null, r -> { Id source = ((HugeVertex) r.get(0)).id(); Id target = ((HugeVertex) r.get(1)).id(); @@ -91,7 +90,7 @@ public List multiNodeShortestPathConcurrent(List> pairs, public List multiNodeShortestPathSingle(List> pairs, EdgeStep step, int maxDepth, long capacity) { - List results = new ArrayList<>(); + List results = newList(); ShortestPathTraverser traverser = new ShortestPathTraverser(this.graph()); for (Pair pair : pairs) { @@ -109,7 +108,7 @@ private static void cmn(List all, int m, int n, int current, assert m <= all.size(); assert current <= all.size(); if (result == null) { - result = new ArrayList<>(n); + result = newList(n); } if (n == 0) { // All n items are selected diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/NeighborRankTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/NeighborRankTraverser.java index 339343e844..e6f21f4493 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/NeighborRankTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/NeighborRankTraverser.java @@ -19,9 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -64,13 +61,13 @@ public List> neighborRank(Id source, List steps) { boolean sameLayerTransfer = true; long access = 0; // Results: ranks of each layer - List ranks = new ArrayList<>(); + List ranks = newList(); ranks.add(Ranks.of(source, 1.0)); for (Step step : steps) { Ranks lastLayerRanks = ranks.get(ranks.size() - 1); - Map sameLayerIncrRanks = new HashMap<>(); - List adjacencies = new ArrayList<>(); + Map sameLayerIncrRanks = newMap(); + List adjacencies = newList(); MultivaluedMap newVertices = newMultivalueMap(); // Traversal vertices of previous level for (Map.Entry> entry : sources.entrySet()) { @@ -79,8 +76,8 @@ public List> neighborRank(Id source, List steps) { step.edgeStep); Adjacencies adjacenciesV = new Adjacencies(vertex); - Set sameLayerNodesV = new HashSet<>(); - Map> prevLayerNodesV = new HashMap<>(); + Set sameLayerNodesV = newIdSet(); + Map> prevLayerNodesV = newMap(); while (edges.hasNext()) { HugeEdge edge = (HugeEdge) edges.next(); Id target = edge.id().otherVertexId(); @@ -166,7 +163,8 @@ private boolean belongToPrevLayers(List ranks, Id target, for (int i = ranks.size() - 2; i > 0; i--) { Ranks prevLayerRanks = ranks.get(i); if (prevLayerRanks.containsKey(target)) { - Set nodes = prevLayerNodes.computeIfAbsent(i, HashSet::new); + Set nodes = prevLayerNodes.computeIfAbsent( + i, HugeTraverser::newSet); nodes.add(target); return true; } @@ -220,7 +218,7 @@ private Ranks contributeNewLayer(List adjacencies, private List> topRanks(List ranks, List steps) { assert ranks.size() > 0; - List> results = new ArrayList<>(ranks.size()); + List> results = newList(ranks.size()); // The first layer is root node so skip i=0 results.add(ranks.get(0)); for (int i = 1; i < ranks.size(); i++) { @@ -264,7 +262,7 @@ private static class Adjacencies { public Adjacencies(Id source) { this.source = source; - this.nodes = new ArrayList<>(); + this.nodes = newList(); this.degree = -1L; } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/OltpTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/OltpTraverser.java index 8dcd9909c7..af4a73cc39 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/OltpTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/OltpTraverser.java @@ -21,14 +21,11 @@ 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.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; @@ -38,8 +35,6 @@ import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.iterator.FilterIterator; -import com.baidu.hugegraph.structure.HugeEdge; -import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.util.Consumers; import jersey.repackaged.com.google.common.base.Objects; @@ -81,48 +76,25 @@ public static void destroy() { } } - protected Set adjacentVertices(Id source, Set latest, - EdgeStep step, Set all, - long remaining, boolean single) { - if (single) { - return this.adjacentVertices(source, 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) { - 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, + boolean concurrent) { + if (concurrent) { + return this.traverseIds(ids, consumer); + } else { + long count = 0L; + while (ids.hasNext()) { + count++; + consumer.accept(ids.next()); + } + return count; + } + } + protected long traverseIds(Iterator ids, Consumer consumer) { return this.traverse(ids, consumer, "traverse-ids"); } 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 14bf14d445..15f33fcc35 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 @@ -20,19 +20,16 @@ package com.baidu.hugegraph.traversal.algorithm; import java.util.Iterator; -import java.util.List; -import java.util.Map; - -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.perf.PerfUtil.Watched; import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.traversal.algorithm.records.PathsRecords; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; -import com.google.common.collect.ImmutableList; public class PathsTraverser extends HugeTraverser { @@ -40,6 +37,7 @@ public PathsTraverser(HugeGraph graph) { super(graph); } + @Watched public PathSet paths(Id sourceV, Directions sourceDir, Id targetV, Directions targetDir, String label, int depth, long degree, long capacity, long limit) { @@ -83,141 +81,93 @@ public PathSet paths(Id sourceV, Directions sourceDir, private class Traverser { - private MultivaluedMap sources = newMultivalueMap(); - private MultivaluedMap targets = newMultivalueMap(); - private MultivaluedMap sourcesAll = newMultivalueMap(); - private MultivaluedMap targetsAll = newMultivalueMap(); + private final PathsRecords record; private final Id label; private final long degree; private final long capacity; private final long limit; + private PathSet paths; public Traverser(Id sourceV, Id targetV, Id label, long degree, long capacity, long limit) { - this.sources.add(sourceV, new Node(sourceV)); - this.targets.add(targetV, new Node(targetV)); - this.sourcesAll.putAll(this.sources); - this.targetsAll.putAll(this.targets); + this.record = new PathsRecords(false, sourceV, targetV); this.label = label; this.degree = degree; this.capacity = capacity; this.limit = limit; + this.paths = new PathSet(); } /** * Search forward from source */ + @Watched public void forward(Directions direction) { - MultivaluedMap newVertices = newMultivalueMap(); Iterator edges; - // Traversal vertices of previous level - for (Map.Entry> entry : this.sources.entrySet()) { - Id vid = entry.getKey(); + + this.record.startOneLayer(true); + while (this.record.hasNextKey()) { + Id vid = this.record.nextKey(); + edges = edgesOfVertex(vid, direction, this.label, this.degree); 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; + PathSet results = this.record.findPath(target, null, + true, false); + for (Path path : results) { + this.paths.add(path); + if (this.reachLimit()) { + return; } - - // 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 sources - this.sources = newVertices; - // Record all passed vertices - for (Map.Entry> entry : newVertices.entrySet()) { - this.sourcesAll.addAll(entry.getKey(), entry.getValue()); - } + this.record.finishOneLayer(); } /** * Search backward from target */ + @Watched public void backward(Directions direction) { - MultivaluedMap newVertices = newMultivalueMap(); Iterator edges; - // Traversal vertices of previous level - for (Map.Entry> entry : this.targets.entrySet()) { - Id vid = entry.getKey(); + + this.record.startOneLayer(false); + while (this.record.hasNextKey()) { + Id vid = this.record.nextKey(); edges = edgesOfVertex(vid, direction, this.label, this.degree); 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; - } - } - } + PathSet results = this.record.findPath(target, null, + true, false); + for (Path path : results) { + this.paths.add(path); + if (this.reachLimit()) { + return; } - - // Add node to next start-nodes - newVertices.add(target, new Node(target, n)); } } } - // Re-init targets - this.targets = newVertices; - // Record all passed vertices - for (Map.Entry> entry : newVertices.entrySet()) { - this.targetsAll.addAll(entry.getKey(), entry.getValue()); - } + this.record.finishOneLayer(); } public PathSet paths() { return this.paths; } - private int accessedNodes() { - return this.sourcesAll.size() + this.targetsAll.size(); - } - private boolean reachLimit() { - checkCapacity(this.capacity, this.accessedNodes(), "paths"); - if (this.limit == NO_LIMIT || this.paths.size() < this.limit) { - return false; - } - return true; + checkCapacity(this.capacity, this.record.accessed(), "paths"); + return this.limit != NO_LIMIT && this.paths.size() >= this.limit; } } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PersonalRankTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PersonalRankTraverser.java index 5c87d61790..d656de559a 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PersonalRankTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/PersonalRankTraverser.java @@ -19,8 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -57,21 +55,21 @@ public Map personalRank(Id source, String label, this.checkVertexExist(source, "source vertex"); E.checkArgumentNotNull(label, "The edge label can't be null"); - Map ranks = new HashMap<>(); + Map ranks = newMap(); ranks.put(source, 1.0); Id labelId = this.graph().edgeLabel(label).id(); Directions dir = this.getStartDirection(source, label); - Set outSeeds = new HashSet<>(); - Set inSeeds = new HashSet<>(); + Set outSeeds = newIdSet(); + Set inSeeds = newIdSet(); if (dir == Directions.OUT) { outSeeds.add(source); } else { inSeeds.add(source); } - Set rootAdjacencies = new HashSet<>(); + Set rootAdjacencies = newIdSet(); for (long i = 0; i < this.maxDepth; i++) { Map newRanks = this.calcNewRanks(outSeeds, inSeeds, labelId, ranks); @@ -93,10 +91,10 @@ public Map personalRank(Id source, String label, private Map calcNewRanks(Set outSeeds, Set inSeeds, Id label, Map ranks) { - Map newRanks = new HashMap<>(); + Map newRanks = newMap(); BiFunction, Directions, Set> neighborIncrRanks; neighborIncrRanks = (seeds, dir) -> { - Set tmpSeeds = new HashSet<>(); + Set tmpSeeds = newIdSet(); for (Id seed : seeds) { Double oldRank = ranks.get(seed); E.checkState(oldRank != null, "Expect rank of seed exists"); 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 17b757c6bf..151928964b 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,9 +19,6 @@ 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; @@ -31,7 +28,9 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.structure.HugeEdge; +import com.baidu.hugegraph.traversal.algorithm.records.ShortestPathRecords; import com.baidu.hugegraph.traversal.algorithm.steps.EdgeStep; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; @@ -43,6 +42,7 @@ public ShortestPathTraverser(HugeGraph graph) { super(graph); } + @Watched public Path shortestPath(Id sourceV, Id targetV, Directions dir, List labels, int depth, long degree, long skipDegree, long capacity) { @@ -60,7 +60,7 @@ public Path shortestPath(Id sourceV, Id targetV, Directions dir, return new Path(ImmutableList.of(sourceV)); } - Map labelMap = new HashMap<>(labels.size()); + Map labelMap = newMap(labels.size()); for (String label : labels) { labelMap.put(this.getEdgeLabelId(label), label); } @@ -73,17 +73,15 @@ public Path shortestPath(Id sourceV, Id targetV, Directions dir, --depth <= 0) { break; } - checkCapacity(traverser.capacity, traverser.size, "shortest path"); + checkCapacity(traverser.capacity, traverser.accessed(), + "shortest path"); if (!(paths = traverser.backward(false)).isEmpty() || --depth <= 0) { - if (!paths.isEmpty()) { - Path path = paths.iterator().next(); - Collections.reverse(path.vertices()); - } break; } - checkCapacity(traverser.capacity, traverser.size, "shortest path"); + checkCapacity(traverser.capacity, traverser.accessed(), + "shortest path"); } return paths.isEmpty() ? Path.EMPTY_PATH : paths.iterator().next(); } @@ -91,7 +89,7 @@ public Path shortestPath(Id sourceV, Id targetV, Directions dir, 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()), + newList(step.labels().values()), depth, step.degree(), step.skipDegree(), capacity); } @@ -115,7 +113,7 @@ public PathSet allShortestPaths(Id sourceV, Id targetV, Directions dir, return paths; } - Map labelMap = new HashMap<>(labels.size()); + Map labelMap = newMap(labels.size()); for (String label : labels) { labelMap.put(this.getEdgeLabelId(label), label); } @@ -127,56 +125,52 @@ public PathSet allShortestPaths(Id sourceV, Id targetV, Directions dir, --depth <= 0) { break; } - checkCapacity(traverser.capacity, traverser.size, "shortest path"); + checkCapacity(traverser.capacity, traverser.accessed(), + "shortest path"); if (!(paths = traverser.backward(true)).isEmpty() || --depth <= 0) { - for (Path path : paths) { - Collections.reverse(path.vertices()); - } break; } - checkCapacity(traverser.capacity, traverser.size, "shortest path"); + checkCapacity(traverser.capacity, traverser.accessed(), + "shortest path"); } return paths; } private class Traverser { - // TODO: change Map to Set to reduce memory cost - private Map sources = newMap(); - private Map targets = newMap(); - + private ShortestPathRecords record; private final Directions direction; 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, Map labels, long degree, long skipDegree, long capacity) { - this.sources.put(sourceV, new Node(sourceV)); - this.targets.put(targetV, new Node(targetV)); + this.record = new ShortestPathRecords(sourceV, targetV); this.direction = dir; this.labels = labels; this.degree = degree; this.skipDegree = skipDegree; this.capacity = capacity; - this.size = 0L; } /** * Search forward from source */ + @Watched public PathSet forward(boolean all) { - PathSet paths = new PathSet(); - Map newVertices = newMap(); + PathSet results = new PathSet(); long degree = this.skipDegree > 0L ? this.skipDegree : this.degree; - // Traversal vertices of previous level - for (Node v : this.sources.values()) { - Iterator edges = edgesOfVertex(v.id(), this.direction, + + this.record.startOneLayer(true); + while (this.record.hasNextKey()) { + Id source = this.record.nextKey(); + + Iterator edges = edgesOfVertex(source, this.direction, this.labels, degree); edges = skipSuperNodeIfNeeded(edges, this.degree, this.skipDegree); @@ -184,50 +178,39 @@ public PathSet forward(boolean all) { HugeEdge edge = (HugeEdge) edges.next(); Id target = edge.id().otherVertexId(); - // If cross point exists, shortest path found, concat them - if (this.targets.containsKey(target)) { - if (this.superNode(target, this.direction)) { - continue; - } - paths.add(new Path( - v.joinPath(this.targets.get(target)))); - if (!all) { - return paths; - } - } + PathSet paths = this.record.findPath(target, + t -> !this.superNode(t, this.direction), + all, false); - /* - * Not found shortest path yet, node is added to - * newVertices if: - * 1. not in sources and newVertices yet - * 2. path of node doesn't have loop - */ - if (!newVertices.containsKey(target) && - !this.sources.containsKey(target) && - !v.contains(target)) { - newVertices.put(target, new Node(target, v)); + if (paths.isEmpty()) { + continue; + } + results.addAll(paths); + if (!all) { + return paths; } } } - // Re-init sources - this.sources = newVertices; - this.size += newVertices.size(); + this.record.finishOneLayer(); - return paths; + return results; } /** * Search backward from target */ + @Watched public PathSet backward(boolean all) { - PathSet paths = new PathSet(); - Map newVertices = newMap(); + PathSet results = new PathSet(); long degree = this.skipDegree > 0L ? this.skipDegree : this.degree; Directions opposite = this.direction.opposite(); - // Traversal vertices of previous level - for (Node v : this.targets.values()) { - Iterator edges = edgesOfVertex(v.id(), opposite, + + this.record.startOneLayer(false); + while (this.record.hasNextKey()) { + Id source = this.record.nextKey(); + + Iterator edges = edgesOfVertex(source, opposite, this.labels, degree); edges = skipSuperNodeIfNeeded(edges, this.degree, this.skipDegree); @@ -235,37 +218,24 @@ public PathSet backward(boolean all) { HugeEdge edge = (HugeEdge) edges.next(); Id target = edge.id().otherVertexId(); - // If cross point exists, shortest path found, concat them - if (this.sources.containsKey(target)) { - if (this.superNode(target, opposite)) { - continue; - } - paths.add(new Path( - v.joinPath(this.sources.get(target)))); - if (!all) { - return paths; - } - } + PathSet paths = this.record.findPath(target, + t -> !this.superNode(t, opposite), + all, false); - /* - * Not found shortest path yet, node is added to - * newVertices if: - * 1. not in targets and newVertices yet - * 2. path of node doesn't have loop - */ - if (!newVertices.containsKey(target) && - !this.targets.containsKey(target) && - !v.contains(target)) { - newVertices.put(target, new Node(target, v)); + if (paths.isEmpty()) { + continue; + } + results.addAll(paths); + if (!all) { + return results; } } } // Re-init targets - this.targets = newVertices; - this.size += newVertices.size(); + this.record.finishOneLayer(); - return paths; + return results; } private boolean superNode(Id vertex, Directions direction) { @@ -276,5 +246,9 @@ private boolean superNode(Id vertex, Directions direction) { this.labels, this.skipDegree); return IteratorUtils.count(edges) >= this.skipDegree; } + + private long accessed() { + return this.record.accessed(); + } } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SingleSourceShortestPathTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SingleSourceShortestPathTraverser.java index 217c687ff7..ce1f4afe24 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SingleSourceShortestPathTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SingleSourceShortestPathTraverser.java @@ -19,9 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -223,7 +220,7 @@ private Iterator skipSuperNodeIfNeeded(Iterator edges) { if (this.skipDegree <= 0L) { return edges; } - List edgeList = new ArrayList<>(); + List edgeList = newList(); int count = 0; while (edges.hasNext()) { if (count < this.degree) { @@ -276,7 +273,7 @@ public static class WeightedPaths extends LinkedHashMap { private static final long serialVersionUID = -313873642177730993L; public Set vertices() { - Set vertices = new HashSet<>(); + Set vertices = newIdSet(); vertices.addAll(this.keySet()); for (NodeWithWeight nw : this.values()) { vertices.addAll(nw.node().path()); @@ -285,7 +282,7 @@ public Set vertices() { } public Map> toMap() { - Map> results = new HashMap<>(); + Map> results = newMap(); for (Map.Entry entry : this.entrySet()) { Id source = entry.getKey(); NodeWithWeight nw = entry.getValue(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SubGraphTraverser.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SubGraphTraverser.java index 7e7f02f75b..87c8032a52 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SubGraphTraverser.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/SubGraphTraverser.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -104,7 +103,7 @@ private class Traverser { private final Id source; private MultivaluedMap sources = newMultivalueMap(); - private Set accessedVertices = newSet(); + private Set accessedVertices = newIdSet(); private final Id label; private int depth; @@ -162,7 +161,7 @@ public PathSet forward(Directions direction) { } int neighborCount = 0; - Set currentNeighbors = new HashSet<>(); + Set currentNeighbors = newIdSet(); while (edges.hasNext()) { neighborCount++; HugeEdge edge = (HugeEdge) edges.next(); 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 3e4bdc5b00..2a3971cedd 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 @@ -19,7 +19,6 @@ package com.baidu.hugegraph.traversal.algorithm; -import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -50,7 +49,7 @@ public Set templatePaths(Iterator sources, checkCapacity(capacity); checkLimit(limit); - List sourceList = new ArrayList<>(); + List sourceList = newList(); while (sources.hasNext()) { sourceList.add(((HugeVertex) sources.next()).id()); } @@ -58,7 +57,7 @@ public Set templatePaths(Iterator sources, 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<>(); + List targetList = newList(); while (targets.hasNext()) { targetList.add(((HugeVertex) targets.next()).id()); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/AbstractRecords.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/AbstractRecords.java new file mode 100644 index 0000000000..e75c94bea6 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/AbstractRecords.java @@ -0,0 +1,64 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.traversal.algorithm.records; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.perf.PerfUtil.Watched; +import com.baidu.hugegraph.traversal.algorithm.records.record.Record; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordFactory; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; +import com.baidu.hugegraph.util.collection.ObjectIntMapping; +import com.baidu.hugegraph.util.collection.MappingFactory; + +public abstract class AbstractRecords implements Records { + + private final ObjectIntMapping idMapping; + private final RecordType type; + private final boolean concurrent; + private Record currentRecord; + + public AbstractRecords(RecordType type, boolean concurrent) { + this.type = type; + this.concurrent = concurrent; + this.idMapping = MappingFactory.newObjectIntMapping(this.concurrent); + } + + @Watched + protected int code(Id id) { + return this.idMapping.object2Code(id); + } + + @Watched + protected Id id(int code) { + return this.idMapping.code2Object(code); + } + + protected Record newRecord() { + return RecordFactory.newRecord(this.type, this.concurrent); + } + + protected Record currentRecord() { + return this.currentRecord; + } + + protected void currentRecord(Record record) { + this.currentRecord = record; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/DoubleWayMultiPathsRecords.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/DoubleWayMultiPathsRecords.java new file mode 100644 index 0000000000..ad76673e33 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/DoubleWayMultiPathsRecords.java @@ -0,0 +1,210 @@ +/* + * 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.records; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Stack; +import java.util.function.Function; + +import org.apache.commons.collections.CollectionUtils; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.perf.PerfUtil.Watched; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.Path; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.PathSet; +import com.baidu.hugegraph.traversal.algorithm.records.record.IntIterator; +import com.baidu.hugegraph.traversal.algorithm.records.record.Record; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; +import com.google.common.collect.Lists; + +public abstract class DoubleWayMultiPathsRecords extends AbstractRecords { + + private final Stack sourceRecords; + private final Stack targetRecords; + + private IntIterator lastRecordKeys; + private int currentKey; + private boolean forward; + private int accessed; + + public DoubleWayMultiPathsRecords(RecordType type, boolean concurrent, + Id sourceV, Id targetV) { + super(type, concurrent); + int sourceCode = this.code(sourceV); + int targetCode = this.code(targetV); + Record firstSourceRecord = this.newRecord(); + Record firstTargetRecord = this.newRecord(); + firstSourceRecord.addPath(sourceCode, 0); + firstTargetRecord.addPath(targetCode, 0); + this.sourceRecords = new Stack<>(); + this.targetRecords = new Stack<>(); + this.sourceRecords.push(firstSourceRecord); + this.targetRecords.push(firstTargetRecord); + + this.accessed = 2; + } + + @Override + public void startOneLayer(boolean forward) { + this.forward = forward; + this.currentRecord(this.newRecord()); + this.lastRecordKeys = this.forward ? this.sourceRecords.peek().keys() : + this.targetRecords.peek().keys(); + } + + @Override + public void finishOneLayer() { + Record record = this.currentRecord(); + if (this.forward) { + this.sourceRecords.push(record); + } else { + this.targetRecords.push(record); + } + this.accessed += record.size(); + } + + @Watched + @Override + public boolean hasNextKey() { + return this.lastRecordKeys.hasNext(); + } + + @Watched + @Override + public Id nextKey() { + this.currentKey = this.lastRecordKeys.next(); + return this.id(this.currentKey); + } + + @Watched + public PathSet findPath(Id target, Function filter, + boolean all, boolean ring) { + assert all; + PathSet results = new PathSet(); + int targetCode = this.code(target); + // If cross point exists, path found, concat them + if (this.forward && this.targetContains(targetCode)) { + results = this.linkPath(this.currentKey, targetCode, ring); + } + if (!this.forward && this.sourceContains(targetCode)) { + results = this.linkPath(targetCode, this.currentKey, ring); + } + this.addPath(targetCode, this.currentKey); + return results; + } + + @Override + public long accessed() { + return this.accessed; + } + + protected boolean sourceContains(int node) { + return this.sourceRecords.peek().containsKey(node); + } + + protected boolean targetContains(int node) { + return this.targetRecords.peek().containsKey(node); + } + + @Watched + private PathSet linkPath(int source, int target, boolean ring) { + PathSet results = new PathSet(); + PathSet sources = this.linkSourcePath(source); + PathSet targets = this.linkTargetPath(target); + for (Path tpath : targets) { + tpath.reverse(); + for (Path spath : sources) { + if (!ring) { + // Avoid loop in path + if (CollectionUtils.containsAny(spath.vertices(), + tpath.vertices())) { + continue; + } + } + List ids = new ArrayList<>(spath.vertices()); + ids.addAll(tpath.vertices()); + Id crosspoint = this.id(this.forward ? target : source); + results.add(new Path(crosspoint, ids)); + } + } + return results; + } + + private PathSet linkSourcePath(int source) { + return this.linkPath(this.sourceRecords, source, + this.sourceRecords.size() - 1); + } + + private PathSet linkTargetPath(int target) { + return this.linkPath(this.targetRecords, target, + this.targetRecords.size() - 1); + } + + private PathSet linkPath(Stack all, int id, int layerIndex) { + PathSet results = new PathSet(); + if (layerIndex == 0) { + Id sid = this.id(id); + results.add(new Path(Lists.newArrayList(sid))); + return results; + } + + Id sid = this.id(id); + Record layer = all.elementAt(layerIndex); + IntIterator iterator = layer.get(id); + while (iterator.hasNext()) { + int parent = iterator.next(); + PathSet paths = this.linkPath(all, parent, layerIndex - 1); + for (Iterator iter = paths.iterator(); iter.hasNext();) { + Path path = iter.next(); + if (path.vertices().contains(sid)) { + iter.remove(); + continue; + } + path.addToLast(sid); + } + + results.addAll(paths); + } + return results; + } + + @Watched + protected void addPath(int current, int parent) { + this.currentRecord().addPath(current, parent); + } + + protected Stack sourceRecords() { + return this.sourceRecords; + } + + protected Stack targetRecords() { + return this.targetRecords; + } + + protected boolean forward() { + return this.forward; + } + + protected int current() { + return this.currentKey; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KneighborRecords.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KneighborRecords.java new file mode 100644 index 0000000000..df7eeb4fa0 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KneighborRecords.java @@ -0,0 +1,73 @@ +/* + * 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.records; + +import static com.baidu.hugegraph.backend.query.Query.NO_LIMIT; + +import java.util.Set; +import java.util.Stack; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.PathSet; +import com.baidu.hugegraph.traversal.algorithm.records.record.IntIterator; +import com.baidu.hugegraph.traversal.algorithm.records.record.Record; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; +import com.baidu.hugegraph.type.define.CollectionType; +import com.baidu.hugegraph.util.collection.CollectionFactory; + +public class KneighborRecords extends SingleWayMultiPathsRecords { + + public KneighborRecords(RecordType type, boolean concurrent, + Id source, boolean nearest) { + super(type, concurrent, source, nearest); + } + + @Override + public int size() { + return (int) this.accessed(); + } + + public Set ids(long limit) { + Set ids = CollectionFactory.newIdSet(CollectionType.EC); + Stack records = this.records(); + // Not include record(i=0) to ignore source vertex + for (int i = 1; i < records.size(); i++) { + IntIterator iterator = records.get(i).keys(); + while ((limit == NO_LIMIT || limit > 0L) && iterator.hasNext()) { + ids.add(this.id(iterator.next())); + limit--; + } + } + return ids; + } + + public PathSet paths(long limit) { + PathSet paths = new PathSet(); + Stack records = this.records(); + for (int i = 1; i < records.size(); i++) { + IntIterator iterator = records.get(i).keys(); + while ((limit == NO_LIMIT || limit > 0L) && iterator.hasNext()) { + paths.add(this.getPath(i, iterator.next())); + limit--; + } + } + return paths; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KoutRecords.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KoutRecords.java new file mode 100644 index 0000000000..da58b6cd3b --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/KoutRecords.java @@ -0,0 +1,65 @@ +/* + * 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.records; + +import static com.baidu.hugegraph.backend.query.Query.NO_LIMIT; + +import java.util.Set; +import java.util.Stack; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.PathSet; +import com.baidu.hugegraph.traversal.algorithm.records.record.IntIterator; +import com.baidu.hugegraph.traversal.algorithm.records.record.Record; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; +import com.baidu.hugegraph.type.define.CollectionType; +import com.baidu.hugegraph.util.collection.CollectionFactory; + +public class KoutRecords extends SingleWayMultiPathsRecords { + + public KoutRecords(RecordType type, boolean concurrent, + Id source, boolean nearest) { + super(type, concurrent, source, nearest); + } + + @Override + public int size() { + return this.currentRecord().size(); + } + + public Set ids(long limit) { + IntIterator iterator = this.records().peek().keys(); + Set ids = CollectionFactory.newIdSet(CollectionType.EC); + while ((limit == NO_LIMIT || limit-- > 0L) && iterator.hasNext()) { + ids.add(this.id(iterator.next())); + } + return ids; + } + + public PathSet paths(long limit) { + PathSet paths = new PathSet(); + Stack records = this.records(); + IntIterator iterator = records.peek().keys(); + while ((limit == NO_LIMIT || limit-- > 0L) && iterator.hasNext()) { + paths.add(this.getPath(records.size() - 1, iterator.next())); + } + return paths; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/PathsRecords.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/PathsRecords.java new file mode 100644 index 0000000000..fd3cb7044b --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/PathsRecords.java @@ -0,0 +1,30 @@ +/* + * 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.records; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; + +public class PathsRecords extends DoubleWayMultiPathsRecords { + + public PathsRecords(boolean concurrent, Id sourceV, Id targetV) { + super(RecordType.ARRAY, concurrent, sourceV, targetV); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/Records.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/Records.java new file mode 100644 index 0000000000..405ed933eb --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/Records.java @@ -0,0 +1,41 @@ +/* + * 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.records; + +import java.util.function.Function; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.PathSet; + +public interface Records { + + public void startOneLayer(boolean forward); + + public void finishOneLayer(); + + public boolean hasNextKey(); + + public Id nextKey(); + + public PathSet findPath(Id target, Function filter, + boolean all, boolean ring); + + public long accessed(); +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/ShortestPathRecords.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/ShortestPathRecords.java new file mode 100644 index 0000000000..9f30bb035d --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/ShortestPathRecords.java @@ -0,0 +1,117 @@ +/* + * 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.records; + +import java.util.ArrayList; +import java.util.List; +import java.util.Stack; +import java.util.function.Function; + +import org.eclipse.collections.impl.map.mutable.primitive.IntIntHashMap; +import org.eclipse.collections.impl.set.mutable.primitive.IntHashSet; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.PathSet; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.Path; +import com.baidu.hugegraph.traversal.algorithm.records.record.Int2IntRecord; +import com.baidu.hugegraph.traversal.algorithm.records.record.Record; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; + +public class ShortestPathRecords extends DoubleWayMultiPathsRecords { + + private final IntHashSet accessedVertices; + private boolean pathFound; + + public ShortestPathRecords(Id sourceV, Id targetV) { + super(RecordType.INT, false, sourceV, targetV); + + this.accessedVertices = new IntHashSet(); + this.accessedVertices.add(this.code(sourceV)); + this.accessedVertices.add(this.code(targetV)); + this.pathFound = false; + } + + @Override + public PathSet findPath(Id target, Function filter, + boolean all, boolean ring) { + assert !ring; + PathSet paths = new PathSet(); + int targetCode = this.code(target); + // If cross point exists, shortest path found, concat them + if (this.forward() && this.targetContains(targetCode) || + !this.forward() && this.sourceContains(targetCode)) { + if (!filter.apply(target)) { + return paths; + } + paths.add(this.forward() ? + this.linkPath(this.current(), targetCode) : + this.linkPath(targetCode, this.current())); + this.pathFound = true; + if (!all) { + return paths; + } + } + /* + * Not found shortest path yet, node is added to + * newVertices if: + * 1. not in sources and newVertices yet + * 2. path of node doesn't have loop + */ + if (!this.pathFound && this.isNew(targetCode)) { + this.addPath(targetCode, this.current()); + } + return paths; + } + + private boolean isNew(int node) { + return !this.currentRecord().containsKey(node) && + !this.accessedVertices.contains(node); + } + + private Path linkPath(int source, int target) { + Path sourcePath = this.linkSourcePath(source); + Path targetPath = this.linkTargetPath(target); + sourcePath.reverse(); + List ids = new ArrayList<>(sourcePath.vertices()); + ids.addAll(targetPath.vertices()); + return new Path(ids); + } + + private Path linkSourcePath(int source) { + return this.linkPath(this.sourceRecords(), source); + } + + private Path linkTargetPath(int target) { + return this.linkPath(this.targetRecords(), target); + } + + private Path linkPath(Stack all, int node) { + int size = all.size(); + List ids = new ArrayList<>(size); + ids.add(this.id(node)); + int value = node; + for (int i = size - 1; i > 0 ; i--) { + IntIntHashMap layer = ((Int2IntRecord) all.elementAt(i)).layer(); + value = layer.get(value); + ids.add(this.id(value)); + } + return new Path(ids); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/SingleWayMultiPathsRecords.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/SingleWayMultiPathsRecords.java new file mode 100644 index 0000000000..2e21dc1cd5 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/SingleWayMultiPathsRecords.java @@ -0,0 +1,176 @@ +/* + * 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.records; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Stack; +import java.util.function.Function; + +import org.eclipse.collections.api.set.primitive.MutableIntSet; +import org.eclipse.collections.impl.map.mutable.primitive.IntIntHashMap; +import org.eclipse.collections.impl.set.mutable.primitive.IntHashSet; + +import com.baidu.hugegraph.HugeException; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.iterator.MapperIterator; +import com.baidu.hugegraph.perf.PerfUtil.Watched; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.Path; +import com.baidu.hugegraph.traversal.algorithm.HugeTraverser.PathSet; +import com.baidu.hugegraph.traversal.algorithm.records.record.Int2IntRecord; +import com.baidu.hugegraph.traversal.algorithm.records.record.IntIterator; +import com.baidu.hugegraph.traversal.algorithm.records.record.Record; +import com.baidu.hugegraph.traversal.algorithm.records.record.RecordType; + +public abstract class SingleWayMultiPathsRecords extends AbstractRecords { + + private final Stack records; + + private final int sourceCode; + private final boolean nearest; + private final MutableIntSet accessedVertices; + + private IntIterator lastRecordKeys; + + public SingleWayMultiPathsRecords(RecordType type, boolean concurrent, + Id source, boolean nearest) { + super(type, concurrent); + + this.nearest = nearest; + + this.sourceCode = this.code(source); + Record firstRecord = this.newRecord(); + firstRecord.addPath(this.sourceCode, 0); + this.records = new Stack<>(); + this.records.push(firstRecord); + + this.accessedVertices = concurrent ? new IntHashSet().asSynchronized() : + new IntHashSet(); + } + + @Override + public void startOneLayer(boolean forward) { + this.currentRecord(this.newRecord()); + this.lastRecordKeys = this.records.peek().keys(); + } + + @Override + public void finishOneLayer() { + this.records.push(this.currentRecord()); + } + + @Override + public boolean hasNextKey() { + return this.lastRecordKeys.hasNext(); + } + + @Override + public Id nextKey() { + return this.id(this.lastRecordKeys.next()); + } + + @Override + public PathSet findPath(Id target, Function filter, + boolean all, boolean ring) { + PathSet paths = new PathSet(); + for (int i = 1; i < this.records.size(); i++) { + IntIterator iterator = this.records.get(i).keys(); + while (iterator.hasNext()) { + paths.add(this.getPath(i, iterator.next())); + } + } + return paths; + } + + @Override + public long accessed() { + return this.accessedVertices.size(); + } + + public Iterator keys() { + return new MapperIterator<>(this.lastRecordKeys, this::id); + } + + @Watched + public void addPath(Id source, Id target) { + int sourceCode = this.code(source); + int targetCode = this.code(target); + if (this.nearest && this.accessedVertices.contains(targetCode) || + !this.nearest && this.currentRecord().containsKey(targetCode) || + targetCode == this.sourceCode) { + return; + } + this.currentRecord().addPath(targetCode, sourceCode); + + this.accessedVertices.add(targetCode); + } + + public abstract int size(); + + public Path getPath(int target) { + List ids = new ArrayList<>(); + for (int i = 0; i < this.records.size(); i++) { + IntIntHashMap layer = ((Int2IntRecord) this.records + .elementAt(i)).layer(); + if (!layer.containsKey(target)) { + continue; + } + + ids.add(this.id(target)); + int parent = layer.get(target); + ids.add(this.id(parent)); + i--; + for (; i > 0; i--) { + layer = ((Int2IntRecord) this.records.elementAt(i)).layer(); + parent = layer.get(parent); + ids.add(this.id(parent)); + } + break; + } + return new Path(ids); + } + + public Path getPath(int layerIndex, int target) { + List ids = new ArrayList<>(); + IntIntHashMap layer = ((Int2IntRecord) this.records + .elementAt(layerIndex)).layer(); + if (!layer.containsKey(target)) { + throw new HugeException("Failed to get path for %s", + this.id(target)); + } + ids.add(this.id(target)); + int parent = layer.get(target); + ids.add(this.id(parent)); + layerIndex--; + for (; layerIndex > 0; layerIndex--) { + layer = ((Int2IntRecord) this.records.elementAt(layerIndex)).layer(); + parent = layer.get(parent); + ids.add(this.id(parent)); + } + Collections.reverse(ids); + return new Path(ids); + } + + public Stack records() { + return this.records; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2ArrayRecord.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2ArrayRecord.java new file mode 100644 index 0000000000..3ec20f9cdd --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2ArrayRecord.java @@ -0,0 +1,56 @@ +/* + * 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.records.record; + +import com.baidu.hugegraph.util.collection.Int2IntsMap; + +public class Int2ArrayRecord implements Record { + + private final Int2IntsMap layer; + + public Int2ArrayRecord() { + this.layer = new Int2IntsMap(); + } + + @Override + public IntIterator keys() { + return new IntIterator(this.layer.keyIterator()); + } + + @Override + public boolean containsKey(int key) { + return this.layer.containsKey(key); + } + + @Override + public IntIterator get(int key) { + return new IntIterator(this.layer.getValues(key)); + } + + @Override + public void addPath(int node, int parent) { + this.layer.add(node, parent); + } + + @Override + public int size() { + return this.layer.size(); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2IntRecord.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2IntRecord.java new file mode 100644 index 0000000000..0b8d56b435 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2IntRecord.java @@ -0,0 +1,60 @@ +/* + * 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.records.record; + +import org.eclipse.collections.impl.map.mutable.primitive.IntIntHashMap; + +public class Int2IntRecord implements Record { + + private final IntIntHashMap layer; + + public Int2IntRecord() { + this.layer = new IntIntHashMap(); + } + + @Override + public IntIterator keys() { + return new IntIterator(this.layer.keySet().intIterator()); + } + + @Override + public boolean containsKey(int key) { + return this.layer.containsKey(key); + } + + @Override + public IntIterator get(int key) { + return null; + } + + @Override + public void addPath(int node, int parent) { + this.layer.put(node, parent); + } + + @Override + public int size() { + return this.layer.size(); + } + + public IntIntHashMap layer() { + return this.layer; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2SetRecord.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2SetRecord.java new file mode 100644 index 0000000000..662e65aaec --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Int2SetRecord.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.traversal.algorithm.records.record; + +import org.eclipse.collections.impl.map.mutable.primitive.IntObjectHashMap; +import org.eclipse.collections.impl.set.mutable.primitive.IntHashSet; + +public class Int2SetRecord implements Record { + + private final IntObjectHashMap layer; + + public Int2SetRecord() { + this.layer = new IntObjectHashMap<>(); + } + + @Override + public IntIterator keys() { + return new IntIterator(this.layer.keySet().intIterator()); + } + + @Override + public boolean containsKey(int key) { + return this.layer.containsKey(key); + } + + @Override + public IntIterator get(int key) { + return new IntIterator(this.layer.get(key).intIterator()); + } + + @Override + public void addPath(int node, int parent) { + if (this.layer.containsKey(node)) { + this.layer.get(node).add(parent); + } else { + this.layer.put(node, IntHashSet.newSetWith(parent)); + } + } + + @Override + public int size() { + return this.layer.size(); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/IntIterator.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/IntIterator.java new file mode 100644 index 0000000000..34fea4d2f8 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/IntIterator.java @@ -0,0 +1,57 @@ +/* + * 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.records.record; + +import java.util.Iterator; + +public class IntIterator implements Iterator { + + private final org.eclipse.collections.api.iterator.IntIterator iterator; + private final int[] array; + private int index; + + public IntIterator(int[] array) { + this.iterator = null; + this.array = array; + this.index = 0; + } + + public IntIterator(org.eclipse.collections.api.iterator.IntIterator intIterator) { + this.iterator = intIterator; + this.array = new int[0]; + this.index = 0; + } + + @Override + public Integer next() { + if (this.iterator != null) { + return this.iterator.next(); + } + return this.array[index++]; + } + + @Override + public boolean hasNext() { + if (this.iterator != null) { + return this.iterator.hasNext(); + } + return this.index < this.array.length; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Record.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Record.java new file mode 100644 index 0000000000..947feae87a --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/Record.java @@ -0,0 +1,33 @@ +/* + * 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.records.record; + +public interface Record { + + public abstract IntIterator keys(); + + public abstract boolean containsKey(int key); + + public abstract IntIterator get(int key); + + public abstract void addPath(int node, int parent); + + public abstract int size(); +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordFactory.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordFactory.java new file mode 100644 index 0000000000..e7141cbf3a --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordFactory.java @@ -0,0 +1,49 @@ +/* + * 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.records.record; + +public class RecordFactory { + + public static Record newRecord(RecordType type) { + return newRecord(type, false); + } + + public static Record newRecord(RecordType type, boolean concurrent) { + Record record; + switch (type) { + case ARRAY: + record = new Int2ArrayRecord(); + break; + case SET: + record = new Int2SetRecord(); + break; + case INT: + record = new Int2IntRecord(); + break; + default: + throw new AssertionError("Unsupported record type: " + type); + } + if (concurrent) { + record = new SyncRecord(record); + } + + return record; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordType.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordType.java new file mode 100644 index 0000000000..8d170184be --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/RecordType.java @@ -0,0 +1,65 @@ +/* + * 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.records.record; + +import com.baidu.hugegraph.type.define.SerialEnum; + +public enum RecordType implements SerialEnum { + + // Java Collection Framework + ARRAY(1, "array"), + + // Eclipse Collection + SET(2, "set"), + + INT(3, "int"); + + private final byte code; + private final String name; + + static { + SerialEnum.register(RecordType.class); + } + + RecordType(int code, String name) { + assert code < 256; + this.code = (byte) code; + this.name = name; + } + + public byte code() { + return this.code; + } + + public String string() { + return this.name; + } + + public static RecordType fromCode(byte code) { + switch (code) { + case 1: + return ARRAY; + case 2: + return SET; + default: + throw new AssertionError("Unsupported record code: " + code); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/SyncRecord.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/SyncRecord.java new file mode 100644 index 0000000000..4882fbddaa --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/traversal/algorithm/records/record/SyncRecord.java @@ -0,0 +1,80 @@ +/* + * 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.records.record; + +public class SyncRecord implements Record { + + private final Object lock; + private final Record record; + + public SyncRecord(Record record) { + this(record, null); + } + + public SyncRecord(Record record, Object newLock) { + if (record == null) { + throw new IllegalArgumentException( + "Cannot create a SyncRecord on a null record"); + } else { + this.record = record; + this.lock = newLock == null ? this : newLock; + } + } + + @Override + public IntIterator keys() { + /* + * Another threads call addPath() will change IntIterator inner array, + * but in kout/kneighbor scenario it's ok because keys() and addPath() + * won't be called simultaneously on same Record. + */ + synchronized (this.lock) { + return this.record.keys(); + } + } + + @Override + public boolean containsKey(int key) { + synchronized (this.lock) { + return this.record.containsKey(key); + } + } + + @Override + public IntIterator get(int key) { + synchronized (this.lock) { + return this.record.get(key); + } + } + + @Override + public void addPath(int node, int parent) { + synchronized (this.lock) { + this.record.addPath(node, parent); + } + } + + @Override + public int size() { + synchronized (this.lock) { + return this.record.size(); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/CollectionType.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/CollectionType.java new file mode 100644 index 0000000000..0d02fbb517 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/CollectionType.java @@ -0,0 +1,67 @@ +/* + * 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.type.define; + +public enum CollectionType implements SerialEnum { + + // Java Collection Framework + JCF(1, "jcf"), + + // Eclipse Collection + EC(2, "ec"), + + // FastUtil + FU(3, "fu"); + + private final byte code; + private final String name; + + static { + SerialEnum.register(CollectionType.class); + } + + CollectionType(int code, String name) { + assert code < 256; + this.code = (byte) code; + this.name = name; + } + + public byte code() { + return this.code; + } + + public String string() { + return this.name; + } + + public static CollectionType fromCode(byte code) { + switch (code) { + case 1: + return JCF; + case 2: + return EC; + case 3: + return FU; + default: + throw new AssertionError( + "Unsupported collection code: " + code); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/CollectionFactory.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/CollectionFactory.java new file mode 100644 index 0000000000..0f98cc8c83 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/CollectionFactory.java @@ -0,0 +1,264 @@ +/* + * 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.util.collection; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.eclipse.collections.api.map.primitive.IntObjectMap; +import org.eclipse.collections.api.map.primitive.MutableIntObjectMap; +import org.eclipse.collections.impl.list.mutable.FastList; +import org.eclipse.collections.impl.map.mutable.UnifiedMap; +import org.eclipse.collections.impl.map.mutable.primitive.IntObjectHashMap; +import org.eclipse.collections.impl.set.mutable.UnifiedSet; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.type.define.CollectionType; +import com.baidu.hugegraph.util.E; + +import it.unimi.dsi.fastutil.objects.Object2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.objects.ObjectArrayList; +import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; + +public class CollectionFactory { + + private final CollectionType type; + + public CollectionFactory() { + this.type = CollectionType.EC; + } + + public CollectionFactory(CollectionType type) { + this.type = type; + } + + public List newList() { + return newList(this.type); + } + + public List newList(int initialCapacity) { + return newList(this.type, initialCapacity); + } + + public List newList(Collection collection) { + return newList(this.type, collection); + } + + public static List newList(CollectionType type) { + switch (type) { + case EC: + return new FastList<>(); + case JCF: + return new ArrayList<>(); + case FU: + return new ObjectArrayList<>(); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public static List newList(CollectionType type, + int initialCapacity) { + switch (type) { + case EC: + return new FastList<>(initialCapacity); + case JCF: + return new ArrayList<>(initialCapacity); + case FU: + return new ObjectArrayList<>(initialCapacity); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public static List newList(CollectionType type, + Collection collection) { + switch (type) { + case EC: + return new FastList<>(collection); + case JCF: + return new ArrayList<>(collection); + case FU: + return new ObjectArrayList<>(collection); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public Set newSet() { + return newSet(this.type); + } + + public Set newSet(int initialCapacity) { + return newSet(this.type, initialCapacity); + } + + public Set newSet(Collection collection) { + return newSet(this.type, collection); + } + + public static Set newSet(CollectionType type) { + switch (type) { + case EC: + return new UnifiedSet<>(); + case JCF: + return new HashSet<>(); + case FU: + return new ObjectOpenHashSet<>(); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public static Set newSet(CollectionType type, + int initialCapacity) { + switch (type) { + case EC: + return new UnifiedSet<>(initialCapacity); + case JCF: + return new HashSet<>(initialCapacity); + case FU: + return new ObjectOpenHashSet<>(initialCapacity); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public static Set newSet(CollectionType type, + Collection collection) { + switch (type) { + case EC: + return new UnifiedSet<>(collection); + case JCF: + return new HashSet<>(collection); + case FU: + return new ObjectOpenHashSet<>(collection); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public Map newMap() { + return newMap(this.type); + } + + public Map newMap(int initialCapacity) { + return newMap(this.type, initialCapacity); + } + + public Map newMap(Map map) { + return newMap(this.type, map); + } + + public static Map newMap(CollectionType type) { + /* + * EC is faster 10%-20% than JCF, and it's more stable & less + * memory cost(size is bigger, EC is better). + */ + switch (type) { + case EC: + return new UnifiedMap<>(); + case JCF: + return new HashMap<>(); + case FU: + return new Object2ObjectOpenHashMap<>(); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public static Map newMap(CollectionType type, + int initialCapacity) { + switch (type) { + case EC: + return new UnifiedMap<>(initialCapacity); + case JCF: + return new HashMap<>(initialCapacity); + case FU: + return new Object2ObjectOpenHashMap<>(initialCapacity); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public static Map newMap(CollectionType type, + Map map) { + switch (type) { + case EC: + return new UnifiedMap<>(map); + case JCF: + return new HashMap<>(map); + case FU: + return new Object2ObjectOpenHashMap<>(map); + default: + throw new AssertionError( + "Unsupported collection type: " + type); + } + } + + public static MutableIntObjectMap newIntObjectMap() { + return new IntObjectHashMap<>(); + } + + public static MutableIntObjectMap newIntObjectMap(int initialCapacity) { + return new IntObjectHashMap<>(initialCapacity); + } + + public static MutableIntObjectMap newIntObjectMap( + IntObjectMap map) { + return new IntObjectHashMap<>(map); + } + + @SuppressWarnings("unchecked") + public static MutableIntObjectMap newIntObjectMap( + Object... objects) { + IntObjectHashMap map = IntObjectHashMap.newMap(); + E.checkArgument(objects.length % 2 == 0, + "Must provide even arguments for " + + "CollectionFactory.newIntObjectMap"); + for (int i = 0; i < objects.length; i += 2) { + int key = objects[i] instanceof Id ? + (int) ((Id) objects[i]).asLong() : (int) objects[i]; + map.put(key, (V) objects[i + 1]); + } + return map; + } + + public IdSet newIdSet() { + return newIdSet(this.type); + } + + public static IdSet newIdSet(CollectionType type) { + return new IdSet(type); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ConcurrentObjectIntMapping.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ConcurrentObjectIntMapping.java new file mode 100644 index 0000000000..e19864c1ff --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ConcurrentObjectIntMapping.java @@ -0,0 +1,46 @@ +/* + * 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.util.collection; + +import com.baidu.hugegraph.perf.PerfUtil.Watched; + +public class ConcurrentObjectIntMapping implements ObjectIntMapping { + + private SingleThreadObjectIntMapping objectIntMapping; + + public ConcurrentObjectIntMapping() { + this.objectIntMapping = new SingleThreadObjectIntMapping<>(); + } + + @Watched + public synchronized int object2Code(Object object) { + return this.objectIntMapping.object2Code(object); + } + + @Watched + public synchronized V code2Object(int code) { + return this.objectIntMapping.code2Object(code); + } + + @Override + public synchronized void clear() { + this.objectIntMapping.clear(); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/IdSet.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/IdSet.java new file mode 100644 index 0000000000..a4937ec55b --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/IdSet.java @@ -0,0 +1,115 @@ +/* + * 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.util.collection; + +import java.util.AbstractSet; +import java.util.Iterator; +import java.util.Set; + +import org.eclipse.collections.api.iterator.MutableLongIterator; +import org.eclipse.collections.impl.set.mutable.primitive.LongHashSet; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.id.IdGenerator; +import com.baidu.hugegraph.iterator.ExtendableIterator; +import com.baidu.hugegraph.type.define.CollectionType; + +public class IdSet extends AbstractSet { + + private final LongHashSet numberIds; + private final Set nonNumberIds; + + public IdSet(CollectionType type) { + this.numberIds = new LongHashSet(); + this.nonNumberIds = CollectionFactory.newSet(type); + } + + @Override + public int size() { + return this.numberIds.size() + this.nonNumberIds.size(); + } + + @Override + public boolean isEmpty() { + return this.numberIds.isEmpty() && this.nonNumberIds.isEmpty(); + } + + public boolean contains(Id id) { + if (id.type() == Id.IdType.LONG) { + return this.numberIds.contains(id.asLong()); + } else { + return this.nonNumberIds.contains(id); + } + } + + @Override + public Iterator iterator() { + return new ExtendableIterator<>( + this.nonNumberIds.iterator(), + new EcIdIterator(this.numberIds.longIterator())); + } + + @Override + public boolean add(Id id) { + if (id.type() == Id.IdType.LONG) { + return this.numberIds.add(id.asLong()); + } else { + return this.nonNumberIds.add(id); + } + } + + public boolean remove(Id id) { + if (id.type() == Id.IdType.LONG) { + return this.numberIds.remove(id.asLong()); + } else { + return this.nonNumberIds.remove(id); + } + } + + @Override + public void clear() { + this.numberIds.clear(); + this.nonNumberIds.clear(); + } + + private static class EcIdIterator implements Iterator { + + private final MutableLongIterator iterator; + + public EcIdIterator(MutableLongIterator iter) { + this.iterator = iter; + } + + @Override + public boolean hasNext() { + return this.iterator.hasNext(); + } + + @Override + public Id next() { + return IdGenerator.of(this.iterator.next()); + } + + @Override + public void remove() { + this.iterator.remove(); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/Int2IntsMap.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/Int2IntsMap.java new file mode 100644 index 0000000000..fd2d2062fa --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/Int2IntsMap.java @@ -0,0 +1,188 @@ +/* + * 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.util.collection; + +import org.eclipse.collections.api.iterator.IntIterator; +import org.eclipse.collections.impl.map.mutable.primitive.IntIntHashMap; + +// TODO: move to common-module +public class Int2IntsMap { + + private static final int INIT_KEY_CAPACITY = 16; + private static final int CHUNK_SIZE = 10; + private static final int EXPANSION_FACTOR = 2; + + private static final int OFFSET_NEXT_FREE = 0; + private static final int OFFSET_SIZE = 1; + private static final int OFFSET_FIRST_CHUNK_DATA = 2; + + /* + * chunkMap chunkTable + * + * -------- --------------- + * | 1 | 0 |--------->0 | 33 | nextFree (free entry pointer) + * | 2 | 10 |-----+ 1 | 19 | size (values count of key) + * | 3 | 40 |---+ | 2 | int data | + * | . | . | | | 3 | int data | + * | x | y | | | . | ... | point to nextFreeChunk + * -------- | | 9 | 20 |-----------------+ + * | | --------------- | + * | +-->10 | 13 | nextFree | + * | 11 | 1 | size | + * | 12 | int data | | + * | 13 | 0 | | + * | . | ... | | + * | 19 | 0 | | + * | --------------- | + * | 20 | int data |<----------------+ + * | 21 | int data | + * | 22 | int data | + * | 23 | int data | + * | . | ... | point to nextFreeChunk + * | 29 | 30 |-----------------+ + * | --------------- | + * | 30 | int data |<----------------+ + * | 31 | int data | + * | 32 | int data | + * | 33 | 0 | + * | . | ... | + * | 39 | 0 | + * | --------------- + * +---->40 | 48 | nextFree + * 41 | 6 | size + * 42 | int data | + * 43 | int data | + * . | ... | + * 47 | int data | + * 48 | 0 | + * 49 | 0 | + * --------------- + * 50 | ... | + * | ... | + * | ... | + * | ... | + */ + + private IntIntHashMap chunkMap; + private int[] chunkTable; + + private int nextFreeChunk; + + public Int2IntsMap() { + this.chunkMap = new IntIntHashMap(INIT_KEY_CAPACITY); + this.chunkTable = new int[INIT_KEY_CAPACITY * CHUNK_SIZE]; + this.nextFreeChunk = 0; + } + + public void add(int key, int value) { + if (this.chunkMap.containsKey(key)) { + int firstChunk = this.chunkMap.get(key); + /* + * The nextFree represent the position where the next element + * will be located. + */ + int nextFree = this.chunkTable[firstChunk + OFFSET_NEXT_FREE]; + if (!this.endOfChunk(nextFree)) { + chunkTable[nextFree] = value; + this.chunkTable[firstChunk + OFFSET_NEXT_FREE]++; + } else { + /* + * If the nextFree points to the end of last chunk, + * allocate a new chunk and let the nextFree point to + * the start of new allocated chunk. + */ + this.ensureCapacity(); + + int lastEntryOfChunk = nextFree; + this.chunkTable[lastEntryOfChunk] = this.nextFreeChunk; + + nextFree = this.nextFreeChunk; + this.chunkTable[nextFree] = value; + this.chunkTable[firstChunk + OFFSET_NEXT_FREE] = nextFree + 1; + + // Update next block + this.nextFreeChunk += CHUNK_SIZE; + } + this.chunkTable[firstChunk + OFFSET_SIZE]++; + } else { + // New key, allocate 1st chunk and init + this.ensureCapacity(); + + // Allocate 1st chunk + this.chunkMap.put(key, this.nextFreeChunk); + + // Init first chunk + int firstChunk = this.nextFreeChunk; + int nextFree = firstChunk + OFFSET_FIRST_CHUNK_DATA; + this.chunkTable[firstChunk + OFFSET_NEXT_FREE] = nextFree + 1; + this.chunkTable[firstChunk + OFFSET_SIZE] = 1; + this.chunkTable[nextFree] = value; + + // Update next block + this.nextFreeChunk += CHUNK_SIZE; + } + } + + public boolean containsKey(int key) { + return this.chunkMap.containsKey(key); + } + + public int[] getValues(int key) { + int firstChunk = this.chunkMap.get(key); + int size = this.chunkTable[firstChunk + OFFSET_SIZE]; + int[] values = new int[size]; + int position = firstChunk + OFFSET_FIRST_CHUNK_DATA; + int i = 0; + while (i < size) { + if (!this.endOfChunk(position)) { + values[i++] = this.chunkTable[position++]; + } else { + position = this.chunkTable[position]; + } + } + return values; + } + + public IntIterator keyIterator() { + return this.chunkMap.keySet().intIterator(); + } + + public int size() { + return this.chunkMap.size(); + } + + private boolean endOfChunk(int position) { + // The last entry of chunk is next chunk pointer + return (position + 1) % CHUNK_SIZE == 0; + } + + private void ensureCapacity() { + if (this.nextFreeChunk >= this.chunkTable.length) { + this.expand(); + } + } + + private void expand() { + int currentSize = this.chunkTable.length; + int[] newTable = new int[currentSize * EXPANSION_FACTOR]; + System.arraycopy(this.chunkTable, 0, newTable, 0, currentSize); + this.chunkTable = newTable; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/MappingFactory.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/MappingFactory.java new file mode 100644 index 0000000000..cc243c8df7 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/MappingFactory.java @@ -0,0 +1,33 @@ +/* + * 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.util.collection; + +public class MappingFactory { + + public static ObjectIntMapping newObjectIntMapping() { + return newObjectIntMapping(false); + } + + public static ObjectIntMapping newObjectIntMapping( + boolean concurrent) { + return concurrent ? new ConcurrentObjectIntMapping<>() : + new SingleThreadObjectIntMapping<>(); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ObjectIntMapping.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ObjectIntMapping.java new file mode 100644 index 0000000000..21737092df --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/ObjectIntMapping.java @@ -0,0 +1,29 @@ +/* + * 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.util.collection; + +public interface ObjectIntMapping { + + public int object2Code(Object object); + + public V code2Object(int code); + + public void clear(); +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/SingleThreadObjectIntMapping.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/SingleThreadObjectIntMapping.java new file mode 100644 index 0000000000..0084d0bac3 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/collection/SingleThreadObjectIntMapping.java @@ -0,0 +1,76 @@ +/* + * 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.util.collection; + +import org.eclipse.collections.impl.map.mutable.primitive.IntObjectHashMap; + +import com.baidu.hugegraph.HugeException; +import com.baidu.hugegraph.perf.PerfUtil.Watched; + +public class SingleThreadObjectIntMapping implements ObjectIntMapping { + + private static final int MAGIC = 1 << 16; + private static final int MAX_OFFSET = 10; + + private final IntObjectHashMap int2IdMap; + + public SingleThreadObjectIntMapping() { + this.int2IdMap = new IntObjectHashMap<>(); + } + + @Watched + @SuppressWarnings("unchecked") + public int object2Code(Object object) { + int code = object.hashCode(); + // TODO: improve hash algorithm + for (int i = 1; i > 0; i <<= 1) { + for (int j = 0; j < MAX_OFFSET; j++) { + V existed = this.int2IdMap.get(code); + if (existed == null) { + this.int2IdMap.put(code, (V) object); + return code; + } + if (existed.equals(object)) { + return code; + } + code = code + i + j; + /* + * If i < MAGIC, try (i * 2) to reduce conflicts, otherwise + * try (i + 1), (i + 2), ..., (i + 10) to try more times + * before try (i * 2). + */ + if (i < MAGIC) { + break; + } + } + } + throw new HugeException("Failed to get code for object: %s", object); + } + + @Watched + public V code2Object(int code) { + return this.int2IdMap.get(code); + } + + @Override + public void clear() { + this.int2IdMap.clear(); + } +} diff --git a/hugegraph-dist/src/main/java/com/baidu/hugegraph/dist/HugeGraphServer.java b/hugegraph-dist/src/main/java/com/baidu/hugegraph/dist/HugeGraphServer.java index 7b0def5ab6..2dfdccea54 100644 --- a/hugegraph-dist/src/main/java/com/baidu/hugegraph/dist/HugeGraphServer.java +++ b/hugegraph-dist/src/main/java/com/baidu/hugegraph/dist/HugeGraphServer.java @@ -88,7 +88,7 @@ public void stop() { } } - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Throwable { if (args.length != 2) { String msg = "HugeGraphServer can only accept two config files"; LOG.error(msg); diff --git a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/ExampleUtil.java b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/ExampleUtil.java index 3c63d1a2dd..35177d017e 100644 --- a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/ExampleUtil.java +++ b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/ExampleUtil.java @@ -87,8 +87,8 @@ public static HugeGraph loadGraph(boolean needClear, boolean needProfile) { public static void profile() { try { PerfUtil.instance().profilePackage("com.baidu.hugegraph"); - } catch (Exception e) { - throw new RuntimeException(e); + } catch (Throwable t) { + throw new RuntimeException(t); } } diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java index b51f49344b..d0aef334aa 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java @@ -21,7 +21,6 @@ import java.util.Collections; import java.util.Date; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; @@ -77,6 +76,7 @@ import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.Events; +import com.baidu.hugegraph.util.collection.CollectionFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -3090,7 +3090,8 @@ public void testQueryAdjacentVerticesOfEdges() { // Fill edge properties Assert.assertEquals(2, edge.getProperties().size()); Whitebox.setInternalState(edge, "propLoaded", false); - Whitebox.setInternalState(edge, "properties", new HashMap<>()); + Whitebox.setInternalState(edge, "properties", + CollectionFactory.newIntObjectMap()); Assert.assertEquals(0, edge.getProperties().size()); Assert.assertEquals(2, edge.getFilledProperties().size()); Assert.assertEquals(2, edge.getProperties().size()); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java index e1e8673a6b..cb89173e74 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java @@ -36,8 +36,11 @@ import com.baidu.hugegraph.unit.core.DataTypeTest; import com.baidu.hugegraph.unit.core.DirectionsTest; import com.baidu.hugegraph.unit.core.ExceptionTest; +import com.baidu.hugegraph.unit.core.IdSetTest; +import com.baidu.hugegraph.unit.core.Int2IntsMapTest; import com.baidu.hugegraph.unit.core.LocksTableTest; import com.baidu.hugegraph.unit.core.PageStateTest; +import com.baidu.hugegraph.unit.core.ObjectIntMappingTest; import com.baidu.hugegraph.unit.core.QueryTest; import com.baidu.hugegraph.unit.core.RangeTest; import com.baidu.hugegraph.unit.core.RolePermissionTest; @@ -61,6 +64,7 @@ import com.baidu.hugegraph.unit.serializer.StoreSerializerTest; import com.baidu.hugegraph.unit.serializer.TableBackendEntryTest; import com.baidu.hugegraph.unit.serializer.TextBackendEntryTest; +import com.baidu.hugegraph.unit.util.CollectionFactoryTest; import com.baidu.hugegraph.unit.util.CompressUtilTest; import com.baidu.hugegraph.unit.util.JsonUtilTest; import com.baidu.hugegraph.unit.util.StringEncodingTest; @@ -103,6 +107,9 @@ BackendStoreSystemInfoTest.class, TraversalUtilTest.class, PageStateTest.class, + Int2IntsMapTest.class, + ObjectIntMappingTest.class, + IdSetTest.class, /* serializer */ BytesBufferTest.class, @@ -129,7 +136,8 @@ VersionTest.class, JsonUtilTest.class, StringEncodingTest.class, - CompressUtilTest.class + CompressUtilTest.class, + CollectionFactoryTest.class }) public class UnitTestSuite { } diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/cache/CacheManagerTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/cache/CacheManagerTest.java index ac06bd04a4..16a911e417 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/cache/CacheManagerTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/cache/CacheManagerTest.java @@ -234,7 +234,7 @@ public void testCacheListModify() { @Test public void testCacheExpire() { Cache cache1 = new RamCache(); - cache1.expire(28 * 1000L); + cache1.expire(26 * 1000L); Cache cache2 = new RamCache(); cache2.expire(0); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/IdSetTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/IdSetTest.java new file mode 100644 index 0000000000..be2af1c0d9 --- /dev/null +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/IdSetTest.java @@ -0,0 +1,328 @@ +/* + * 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.unit.core; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.Random; +import java.util.Set; +import java.util.UUID; + +import org.apache.commons.lang.RandomStringUtils; +import org.eclipse.collections.impl.set.mutable.primitive.LongHashSet; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.id.IdGenerator; +import com.baidu.hugegraph.testutil.Assert; +import com.baidu.hugegraph.testutil.Whitebox; +import com.baidu.hugegraph.type.define.CollectionType; +import com.baidu.hugegraph.util.collection.IdSet; + +public class IdSetTest { + + private static final int SIZE = 10000; + + private static IdSet idSet; + + @Before + public void setup() { + // pass + } + + @After + public void teardown() { + // pass + } + + @Test + public void testIdSetWithOnlyNumberId() { + Random random = new Random(); + Set numbers = new HashSet<>(); + for (CollectionType type : CollectionType.values()) { + idSet = new IdSet(type); + for (int i = 1; i < SIZE; i++) { + long number = random.nextLong(); + numbers.add(number); + idSet.add(IdGenerator.of(number)); + } + Assert.assertEquals(numbers.size(), idSet.size()); + + LongHashSet numberIds = Whitebox.getInternalState(idSet, + "numberIds"); + Assert.assertEquals(numbers.size(), numberIds.size()); + Set nonNumberIds = Whitebox.getInternalState(idSet, + "nonNumberIds"); + Assert.assertTrue(nonNumberIds.isEmpty()); + + numbers.clear(); + idSet.clear(); + } + } + + @Test + public void testIdSetWithOnlyUUIDId() { + Set uuids = new HashSet<>(); + for (CollectionType type : CollectionType.values()) { + idSet = new IdSet(type); + for (int i = 0; i < SIZE; i++) { + UUID uuid = UUID.randomUUID(); + uuids.add(uuid); + idSet.add(IdGenerator.of(uuid)); + } + Assert.assertEquals(uuids.size(), idSet.size()); + + LongHashSet numberIds = Whitebox.getInternalState(idSet, + "numberIds"); + Assert.assertTrue(numberIds.isEmpty()); + Set nonNumberIds = Whitebox.getInternalState(idSet, + "nonNumberIds"); + Assert.assertEquals(uuids.size(), nonNumberIds.size()); + + uuids.clear(); + idSet.clear(); + } + } + + + @Test + public void testIdSetWithOnlyStringId() { + Set strings = new HashSet<>(); + for (CollectionType type : CollectionType.values()) { + idSet = new IdSet(type); + for (int i = 0; i < SIZE; i++) { + String string = RandomStringUtils.randomAlphanumeric(10); + strings.add(string); + idSet.add(IdGenerator.of(string)); + } + Assert.assertEquals(strings.size(), idSet.size()); + + LongHashSet numberIds = Whitebox.getInternalState(idSet, + "numberIds"); + Assert.assertTrue(numberIds.isEmpty()); + Set nonNumberIds = Whitebox.getInternalState(idSet, + "nonNumberIds"); + Assert.assertEquals(strings.size(), nonNumberIds.size()); + + strings.clear(); + idSet.clear(); + } + } + + @Test + public void testIdSetWithMixedId() { + Random random = new Random(); + Set numbers = new HashSet<>(); + Set uuids = new HashSet<>(); + Set strings = new HashSet<>(); + for (CollectionType type : CollectionType.values()) { + idSet = new IdSet(type); + for (int i = 1; i < SIZE; i++) { + long number = random.nextLong(); + numbers.add(number); + idSet.add(IdGenerator.of(number)); + } + for (int i = 0; i < SIZE; i++) { + UUID uuid = UUID.randomUUID(); + uuids.add(uuid); + idSet.add(IdGenerator.of(uuid)); + } + for (int i = 0; i < SIZE; i++) { + String string = RandomStringUtils.randomAlphanumeric(10); + strings.add(string); + idSet.add(IdGenerator.of(string)); + } + Assert.assertEquals(numbers.size() + uuids.size() + strings.size(), + idSet.size()); + + LongHashSet numberIds = Whitebox.getInternalState(idSet, + "numberIds"); + Assert.assertEquals(numbers.size(), numberIds.size()); + Set nonNumberIds = Whitebox.getInternalState(idSet, + "nonNumberIds"); + Assert.assertEquals(uuids.size() + strings.size(), + nonNumberIds.size()); + + numbers.clear(); + uuids.clear(); + strings.clear(); + idSet.clear(); + } + } + + @Test + public void testIdSetContains() { + Random random = new Random(); + Set numbers = new HashSet<>(); + Set uuids = new HashSet<>(); + Set strings = new HashSet<>(); + long number = 0L; + UUID uuid = null; + String string = null; + for (CollectionType type : CollectionType.values()) { + idSet = new IdSet(type); + for (int i = 1; i < SIZE; i++) { + number = random.nextLong(); + numbers.add(number); + idSet.add(IdGenerator.of(number)); + } + for (int i = 0; i < SIZE; i++) { + uuid = UUID.randomUUID(); + uuids.add(uuid); + idSet.add(IdGenerator.of(uuid)); + } + for (int i = 0; i < SIZE; i++) { + string = RandomStringUtils.randomAlphanumeric(10); + strings.add(string); + idSet.add(IdGenerator.of(string)); + } + Assert.assertEquals(numbers.size() + uuids.size() + strings.size(), + idSet.size()); + + LongHashSet numberIds = Whitebox.getInternalState(idSet, + "numberIds"); + Assert.assertEquals(numbers.size(), numberIds.size()); + Set nonNumberIds = Whitebox.getInternalState(idSet, + "nonNumberIds"); + Assert.assertEquals(uuids.size() + strings.size(), + nonNumberIds.size()); + + Assert.assertTrue(idSet.contains(IdGenerator.of(number))); + Assert.assertTrue(idSet.contains(IdGenerator.of(uuid))); + Assert.assertTrue(idSet.contains(IdGenerator.of(string))); + + numbers.clear(); + uuids.clear(); + strings.clear(); + idSet.clear(); + } + } + + @Test + public void testIdSetIterator() { + Random random = new Random(); + Set numbers = new HashSet<>(); + Set uuids = new HashSet<>(); + Set strings = new HashSet<>(); + for (CollectionType type : CollectionType.values()) { + idSet = new IdSet(type); + for (int i = 1; i < SIZE; i++) { + long number = random.nextLong(); + numbers.add(number); + idSet.add(IdGenerator.of(number)); + } + for (int i = 0; i < SIZE; i++) { + UUID uuid = UUID.randomUUID(); + uuids.add(uuid); + idSet.add(IdGenerator.of(uuid)); + } + for (int i = 0; i < SIZE; i++) { + String string = RandomStringUtils.randomAlphanumeric(10); + strings.add(string); + idSet.add(IdGenerator.of(string)); + } + Assert.assertEquals(numbers.size() + uuids.size() + strings.size(), + idSet.size()); + + LongHashSet numberIds = Whitebox.getInternalState(idSet, + "numberIds"); + Assert.assertEquals(numbers.size(), numberIds.size()); + Set nonNumberIds = Whitebox.getInternalState(idSet, + "nonNumberIds"); + Assert.assertEquals(uuids.size() + strings.size(), + nonNumberIds.size()); + + Iterator iterator = idSet.iterator(); + while (iterator.hasNext()) { + Id id = iterator.next(); + if (id instanceof IdGenerator.LongId) { + Assert.assertTrue(numbers.contains(id.asLong())); + } else if (id instanceof IdGenerator.UuidId) { + Assert.assertTrue(id.uuid() && + uuids.contains(id.asObject())); + } else { + Assert.assertTrue(id instanceof IdGenerator.StringId); + Assert.assertTrue(strings.contains(id.asString())); + } + } + + numbers.clear(); + uuids.clear(); + strings.clear(); + idSet.clear(); + } + } + + @Test + public void testIdSetRemove() { + Random random = new Random(); + Set numbers = new HashSet<>(); + Set uuids = new HashSet<>(); + Set strings = new HashSet<>(); + for (CollectionType type : CollectionType.values()) { + idSet = new IdSet(type); + for (int i = 1; i < SIZE; i++) { + long number = random.nextLong(); + numbers.add(number); + idSet.add(IdGenerator.of(number)); + } + for (int i = 0; i < SIZE; i++) { + UUID uuid = UUID.randomUUID(); + uuids.add(uuid); + idSet.add(IdGenerator.of(uuid)); + } + for (int i = 0; i < SIZE; i++) { + String string = RandomStringUtils.randomAlphanumeric(10); + strings.add(string); + idSet.add(IdGenerator.of(string)); + } + Assert.assertEquals(numbers.size() + uuids.size() + strings.size(), + idSet.size()); + + LongHashSet numberIds = Whitebox.getInternalState(idSet, + "numberIds"); + Assert.assertEquals(numbers.size(), numberIds.size()); + Set nonNumberIds = Whitebox.getInternalState(idSet, + "nonNumberIds"); + Assert.assertEquals(uuids.size() + strings.size(), + nonNumberIds.size()); + + for (long number : numbers) { + idSet.remove(IdGenerator.of(number)); + } + Assert.assertEquals(nonNumberIds.size(), idSet.size()); + + for (UUID uuid : uuids) { + idSet.remove(IdGenerator.of(uuid)); + } + for (String string : strings) { + idSet.remove(IdGenerator.of(string)); + } + Assert.assertTrue(idSet.isEmpty()); + + numbers.clear(); + uuids.clear(); + strings.clear(); + idSet.clear(); + } + } +} diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/Int2IntsMapTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/Int2IntsMapTest.java new file mode 100644 index 0000000000..a8573565a0 --- /dev/null +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/Int2IntsMapTest.java @@ -0,0 +1,177 @@ +/* + * 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.unit.core; + +import java.util.HashSet; +import java.util.Random; +import java.util.Set; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.baidu.hugegraph.testutil.Assert; +import com.baidu.hugegraph.util.collection.Int2IntsMap; + +public class Int2IntsMapTest { + + private static final int KEY_NUMBER = 5; + private static final int VALUE_NUMBER = 1000; + + @Before + public void setup() { + // pass + } + + @After + public void teardown() { + // pass + } + + @Test + public void testInt2IntsMap() { + Int2IntsMap map = new Int2IntsMap(); + + int i = 100; + int j = 200; + int k = 250; + int l = 255; + int m = 270; + Random random = new Random(); + for (int n = 0; n < VALUE_NUMBER; n++) { + switch (random.nextInt() % KEY_NUMBER) { + case 0: + if (i < 200) { + map.add(1, i++); + } + break; + case 1: + if (j < 250) { + map.add(2, j++); + } + break; + case 2: + if (k < 255) { + map.add(3, k++); + } + break; + case 3: + if (l < 270) { + map.add(4, l++); + } + break; + case 4: + if (m < 300) { + map.add(5, m++); + } + break; + } + } + + int[][] results = new int[KEY_NUMBER][]; + results[0] = new int[]{100,101,102,103,104,105,106,107,108,109, + 110,111,112,113,114,115,116,117,118,119, + 120,121,122,123,124,125,126,127,128,129, + 130,131,132,133,134,135,136,137,138,139, + 140,141,142,143,144,145,146,147,148,149, + 150,151,152,153,154,155,156,157,158,159, + 160,161,162,163,164,165,166,167,168,169, + 170,171,172,173,174,175,176,177,178,179, + 180,181,182,183,184,185,186,187,188,189, + 190,191,192,193,194,195,196,197,198,199}; + results[1] = new int[]{200,201,202,203,204,205,206,207,208,209, + 210,211,212,213,214,215,216,217,218,219, + 220,221,222,223,224,225,226,227,228,229, + 230,231,232,233,234,235,236,237,238,239, + 240,241,242,243,244,245,246,247,248,249}; + results[2] = new int[]{250,251,252,253,254}; + results[3] = new int[]{255,256,257,258,259,260,261,262, + 263,264,265,266,267,268,269}; + results[4] = new int[]{270,271,272,273,274,275,276,277,278,279, + 280,281,282,283,284,285,286,287,288,289, + 290,291,292,293,294,295,296,297,298,299}; + + for (int ii = 0; ii < KEY_NUMBER; ii++) { + int[] result = map.getValues(ii + 1); + Assert.assertArrayEquals(results[ii], result); + } + } + + @Test + public void testInt2IntsMapRandom() { + Int2IntsMap map = new Int2IntsMap(); + + Random random = new Random(); + for (int i = 0; i < VALUE_NUMBER; i++) { + map.add(Math.abs(random.nextInt()) % KEY_NUMBER, i); + } + + Set all = new HashSet<>(); + for (int i = 0; i < KEY_NUMBER; i++) { + int[] values = map.getValues(i); + for (int j : values) { + all.add(j); + } + } + Assert.assertEquals(VALUE_NUMBER, all.size()); + } + + @Test + public void testInt2IntsMapContainsKey() { + Int2IntsMap map = new Int2IntsMap(); + + Random random = new Random(); + for (int i = 0; i < VALUE_NUMBER; i++) { + map.add(Math.abs(random.nextInt()) % KEY_NUMBER, i); + } + + Set all = new HashSet<>(); + for (int i = 0; i < KEY_NUMBER; i++) { + int[] values = map.getValues(i); + for (int j : values) { + all.add(j); + } + } + Assert.assertEquals(VALUE_NUMBER, all.size()); + for (int i = 0; i < 5; i++) { + Assert.assertTrue(map.containsKey(i)); + } + } + + @Test + public void testInt2IntsMapSize() { + Int2IntsMap map = new Int2IntsMap(); + + Random random = new Random(); + for (int i = 0; i < VALUE_NUMBER; i++) { + map.add(Math.abs(random.nextInt()) % KEY_NUMBER, i); + } + + Set all = new HashSet<>(); + for (int i = 0; i < KEY_NUMBER; i++) { + int[] values = map.getValues(i); + for (int j : values) { + all.add(j); + } + } + Assert.assertEquals(VALUE_NUMBER, all.size()); + Assert.assertEquals(KEY_NUMBER, map.size()); + } +} diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/ObjectIntMappingTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/ObjectIntMappingTest.java new file mode 100644 index 0000000000..85181d266d --- /dev/null +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/ObjectIntMappingTest.java @@ -0,0 +1,143 @@ +/* + * 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.unit.core; + +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.Set; +import java.util.UUID; + +import org.apache.commons.lang.RandomStringUtils; +import org.junit.After; +import org.junit.Test; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.id.IdGenerator; +import com.baidu.hugegraph.testutil.Assert; +import com.baidu.hugegraph.util.collection.MappingFactory; +import com.baidu.hugegraph.util.collection.ObjectIntMapping; + +public class ObjectIntMappingTest { + + private static final int OBJECT_NUMBER = 1000000; + private static ObjectIntMapping mapping = + MappingFactory.newObjectIntMapping(); + + @After + public void clear() { + mapping.clear(); + } + + @Test + public void testNumberIdMapping() { + Set codes = new LinkedHashSet<>(); + for (int i = 0; i < OBJECT_NUMBER; i++) { + codes.add(mapping.object2Code(IdGenerator.of(i))); + } + Assert.assertEquals(OBJECT_NUMBER, codes.size()); + + int j = 0; + for (Integer code : codes) { + Assert.assertEquals(IdGenerator.of(j++), mapping.code2Object(code)); + } + } + + @Test + public void testStringIdMapping() { + Set strings = new LinkedHashSet<>(); + Set codes = new LinkedHashSet<>(); + for (int i = 0; i < OBJECT_NUMBER; i++) { + String string = RandomStringUtils.randomAlphanumeric(10); + strings.add(string); + codes.add(mapping.object2Code(IdGenerator.of(string))); + } + Assert.assertEquals(strings.size(), codes.size()); + + Iterator strIter = strings.iterator(); + Iterator codeIter = codes.iterator(); + while (strIter.hasNext() && codeIter.hasNext()) { + Assert.assertEquals(IdGenerator.of(strIter.next()), + mapping.code2Object(codeIter.next())); + } + + Assert.assertFalse(strIter.hasNext()); + Assert.assertFalse(codeIter.hasNext()); + } + + @Test + public void testUUIDIdMapping() { + Set uuids = new LinkedHashSet<>(); + Set codes = new LinkedHashSet<>(); + for (int i = 0; i < OBJECT_NUMBER; i++) { + UUID uuid = UUID.randomUUID(); + uuids.add(uuid); + codes.add(mapping.object2Code(IdGenerator.of(uuid))); + } + Assert.assertEquals(uuids.size(), codes.size()); + + Iterator uuidIter = uuids.iterator(); + Iterator codeIter = codes.iterator(); + while (uuidIter.hasNext() && codeIter.hasNext()) { + Assert.assertEquals(IdGenerator.of(uuidIter.next()), + mapping.code2Object(codeIter.next())); + } + + Assert.assertFalse(uuidIter.hasNext()); + Assert.assertFalse(codeIter.hasNext()); + } + + @Test + public void testMixedIdMapping() { + Set codes = new LinkedHashSet<>(); + Set objects = new LinkedHashSet<>(); + Object object; + + for (int i = 0; i < OBJECT_NUMBER; i++) { + object = IdGenerator.of(i); + objects.add(object); + codes.add(mapping.object2Code(object)); + } + + for (int i = 0; i < OBJECT_NUMBER; i++) { + String string = RandomStringUtils.randomAlphanumeric(10); + object = IdGenerator.of(string); + objects.add(object); + codes.add(mapping.object2Code(object)); + } + + for (int i = 0; i < OBJECT_NUMBER; i++) { + UUID uuid = UUID.randomUUID(); + object = IdGenerator.of(uuid); + objects.add(object); + codes.add(mapping.object2Code(object)); + } + Assert.assertEquals(objects.size(), codes.size()); + + Iterator objectIter = objects.iterator(); + Iterator codeIter = codes.iterator(); + while (objectIter.hasNext() && codeIter.hasNext()) { + Assert.assertEquals(objectIter.next(), + mapping.code2Object(codeIter.next())); + } + + Assert.assertFalse(objectIter.hasNext()); + Assert.assertFalse(codeIter.hasNext()); + } +} diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/CollectionFactoryTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/CollectionFactoryTest.java new file mode 100644 index 0000000000..80aa061713 --- /dev/null +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/CollectionFactoryTest.java @@ -0,0 +1,387 @@ +/* + * 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.unit.util; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.eclipse.collections.api.map.primitive.MutableIntObjectMap; +import org.eclipse.collections.impl.list.mutable.FastList; +import org.eclipse.collections.impl.map.mutable.UnifiedMap; +import org.eclipse.collections.impl.map.mutable.primitive.IntObjectHashMap; +import org.eclipse.collections.impl.set.mutable.UnifiedSet; +import org.eclipse.collections.impl.set.mutable.primitive.LongHashSet; +import org.junit.Test; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.testutil.Assert; +import com.baidu.hugegraph.testutil.Whitebox; +import com.baidu.hugegraph.type.define.CollectionType; +import com.baidu.hugegraph.util.collection.CollectionFactory; +import com.baidu.hugegraph.util.collection.IdSet; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; + +import it.unimi.dsi.fastutil.objects.Object2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.objects.ObjectArrayList; +import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; + +public class CollectionFactoryTest { + + private static CollectionFactory factory; + + @Test + public void testCollectionFactoryConstructor() { + factory = new CollectionFactory(); + CollectionType type = Whitebox.getInternalState(factory, "type"); + Assert.assertEquals(CollectionType.EC, type); + + factory = new CollectionFactory(CollectionType.EC); + type = Whitebox.getInternalState(factory, "type"); + Assert.assertEquals(CollectionType.EC, type); + + factory = new CollectionFactory(CollectionType.FU); + type = Whitebox.getInternalState(factory, "type"); + Assert.assertEquals(CollectionType.FU, type); + + factory = new CollectionFactory(CollectionType.JCF); + type = Whitebox.getInternalState(factory, "type"); + Assert.assertEquals(CollectionType.JCF, type); + } + + @Test + public void testNewList() { + // With type + factory = new CollectionFactory(); + List list = factory.newList(); + Assert.assertInstanceOf(FastList.class, list); + + factory = new CollectionFactory(CollectionType.EC); + list = factory.newList(); + Assert.assertInstanceOf(FastList.class, list); + + factory = new CollectionFactory(CollectionType.FU); + list = factory.newList(); + Assert.assertInstanceOf(ObjectArrayList.class, list); + + factory = new CollectionFactory(CollectionType.JCF); + list = factory.newList(); + Assert.assertInstanceOf(ArrayList.class, list); + + // With initial capacity + int initialCapacity = 10; + + factory = new CollectionFactory(); + list = factory.newList(initialCapacity); + Assert.assertInstanceOf(FastList.class, list); + Object[] items = Whitebox.getInternalState(list, "items"); + Assert.assertEquals(initialCapacity, items.length); + + factory = new CollectionFactory(CollectionType.EC); + list = factory.newList(initialCapacity); + Assert.assertInstanceOf(FastList.class, list); + items = Whitebox.getInternalState(list, "items"); + Assert.assertEquals(initialCapacity, items.length); + + factory = new CollectionFactory(CollectionType.FU); + list = factory.newList(initialCapacity); + Assert.assertInstanceOf(ObjectArrayList.class, list); + items = Whitebox.getInternalState(list, "a"); + Assert.assertEquals(initialCapacity, items.length); + + factory = new CollectionFactory(CollectionType.JCF); + list = factory.newList(initialCapacity); + Assert.assertInstanceOf(ArrayList.class, list); + items = Whitebox.getInternalState(list, "elementData"); + Assert.assertEquals(initialCapacity, items.length); + + // With collection + Collection integers = ImmutableSet.of(1, 2, 3, 4); + + factory = new CollectionFactory(); + list = factory.newList(integers); + Assert.assertInstanceOf(FastList.class, list); + Assert.assertEquals(integers.size(), list.size()); + for (int integer : integers) { + Assert.assertTrue(list.contains(integer)); + } + + factory = new CollectionFactory(CollectionType.EC); + list = factory.newList(integers); + Assert.assertInstanceOf(FastList.class, list); + Assert.assertEquals(integers.size(), list.size()); + for (int integer : integers) { + Assert.assertTrue(list.contains(integer)); + } + + factory = new CollectionFactory(CollectionType.FU); + list = factory.newList(integers); + Assert.assertInstanceOf(ObjectArrayList.class, list); + Assert.assertEquals(integers.size(), list.size()); + for (int integer : integers) { + Assert.assertTrue(list.contains(integer)); + } + + factory = new CollectionFactory(CollectionType.JCF); + list = factory.newList(integers); + Assert.assertInstanceOf(ArrayList.class, list); + Assert.assertEquals(integers.size(), list.size()); + for (int integer : integers) { + Assert.assertTrue(list.contains(integer)); + } + } + + @Test + public void testNewSet() { + // With type + factory = new CollectionFactory(); + Set set = factory.newSet(); + Assert.assertInstanceOf(UnifiedSet.class, set); + + factory = new CollectionFactory(CollectionType.EC); + set = factory.newSet(); + Assert.assertInstanceOf(UnifiedSet.class, set); + + factory = new CollectionFactory(CollectionType.FU); + set = factory.newSet(); + Assert.assertInstanceOf(ObjectOpenHashSet.class, set); + + factory = new CollectionFactory(CollectionType.JCF); + set = factory.newSet(); + Assert.assertInstanceOf(HashSet.class, set); + + // With initial capacity + int initialCapacity = 10; + + factory = new CollectionFactory(); + set = factory.newSet(initialCapacity); + Assert.assertInstanceOf(UnifiedSet.class, set); + Object[] items = Whitebox.getInternalState(set, "table"); + // Initial size of UnifiedSet is (initialCapacity / 0.75) + Assert.assertEquals(16, items.length); + + factory = new CollectionFactory(CollectionType.EC); + set = factory.newSet(initialCapacity); + Assert.assertInstanceOf(UnifiedSet.class, set); + items = Whitebox.getInternalState(set, "table"); + // Initial size of UnifiedSet is (initialCapacity / 0.75) + Assert.assertEquals(16, items.length); + + factory = new CollectionFactory(CollectionType.FU); + set = factory.newSet(initialCapacity); + Assert.assertInstanceOf(ObjectOpenHashSet.class, set); + items = Whitebox.getInternalState(set, "key"); + Assert.assertEquals(17, items.length); + + factory = new CollectionFactory(CollectionType.JCF); + set = factory.newSet(initialCapacity); + Assert.assertInstanceOf(HashSet.class, set); + Map map = Whitebox.getInternalState(set, "map"); + Assert.assertInstanceOf(HashMap.class, map); + Assert.assertEquals(0, map.size()); + + // With collection + Collection integers = ImmutableSet.of(1, 2, 3, 4); + + factory = new CollectionFactory(); + set = factory.newSet(integers); + Assert.assertInstanceOf(UnifiedSet.class, set); + Assert.assertEquals(integers.size(), set.size()); + for (int integer : integers) { + Assert.assertTrue(set.contains(integer)); + } + + factory = new CollectionFactory(CollectionType.EC); + set = factory.newSet(integers); + Assert.assertInstanceOf(UnifiedSet.class, set); + Assert.assertEquals(integers.size(), set.size()); + for (int integer : integers) { + Assert.assertTrue(set.contains(integer)); + } + + factory = new CollectionFactory(CollectionType.FU); + set = factory.newSet(integers); + Assert.assertInstanceOf(ObjectOpenHashSet.class, set); + Assert.assertEquals(integers.size(), set.size()); + for (int integer : integers) { + Assert.assertTrue(set.contains(integer)); + } + + factory = new CollectionFactory(CollectionType.JCF); + set = factory.newSet(integers); + Assert.assertInstanceOf(HashSet.class, set); + Assert.assertEquals(integers.size(), set.size()); + for (int integer : integers) { + Assert.assertTrue(set.contains(integer)); + } + } + + @Test + public void testNewMap() { + // With type + factory = new CollectionFactory(); + Map map = factory.newMap(); + Assert.assertInstanceOf(UnifiedMap.class, map); + + factory = new CollectionFactory(CollectionType.EC); + map = factory.newMap(); + Assert.assertInstanceOf(UnifiedMap.class, map); + + factory = new CollectionFactory(CollectionType.FU); + map = factory.newMap(); + Assert.assertInstanceOf(Object2ObjectOpenHashMap.class, map); + + factory = new CollectionFactory(CollectionType.JCF); + map = factory.newMap(); + Assert.assertInstanceOf(HashMap.class, map); + + // With initial capacity + int initialCapacity = 10; + + factory = new CollectionFactory(); + map = factory.newMap(initialCapacity); + Assert.assertInstanceOf(UnifiedMap.class, map); + Object[] items = Whitebox.getInternalState(map, "table"); + // Initial size of UnifiedSet is (initialCapacity / 0.75) * 2 + Assert.assertEquals(32, items.length); + + factory = new CollectionFactory(CollectionType.EC); + map = factory.newMap(initialCapacity); + Assert.assertInstanceOf(UnifiedMap.class, map); + items = Whitebox.getInternalState(map, "table"); + // Initial size of UnifiedSet is (initialCapacity / 0.75) * 2 + Assert.assertEquals(32, items.length); + + factory = new CollectionFactory(CollectionType.FU); + map = factory.newMap(initialCapacity); + Assert.assertInstanceOf(Object2ObjectOpenHashMap.class, map); + items = Whitebox.getInternalState(map, "key"); + Assert.assertEquals(17, items.length); + items = Whitebox.getInternalState(map, "value"); + Assert.assertEquals(17, items.length); + + factory = new CollectionFactory(CollectionType.JCF); + map = factory.newMap(initialCapacity); + Assert.assertInstanceOf(HashMap.class, map); + items = Whitebox.getInternalState(map, "table"); + Assert.assertNull(items); + + // With collection + Map keyValues = ImmutableMap.of(1, "A", 2, "B", + 3, "C", 4, "D"); + + factory = new CollectionFactory(); + map = factory.newMap(keyValues); + Assert.assertInstanceOf(UnifiedMap.class, map); + Assert.assertEquals(keyValues.size(), map.size()); + for (Map.Entry entry : keyValues.entrySet()) { + Assert.assertEquals(entry.getValue(), map.get(entry.getKey())); + } + + factory = new CollectionFactory(CollectionType.EC); + map = factory.newMap(keyValues); + Assert.assertInstanceOf(UnifiedMap.class, map); + Assert.assertEquals(keyValues.size(), map.size()); + for (Map.Entry entry : keyValues.entrySet()) { + Assert.assertEquals(entry.getValue(), map.get(entry.getKey())); + } + + factory = new CollectionFactory(CollectionType.FU); + map = factory.newMap(keyValues); + Assert.assertInstanceOf(Object2ObjectOpenHashMap.class, map); + Assert.assertEquals(keyValues.size(), map.size()); + for (Map.Entry entry : keyValues.entrySet()) { + Assert.assertEquals(entry.getValue(), map.get(entry.getKey())); + } + + factory = new CollectionFactory(CollectionType.JCF); + map = factory.newMap(keyValues); + Assert.assertInstanceOf(HashMap.class, map); + Assert.assertEquals(keyValues.size(), map.size()); + for (Map.Entry entry : keyValues.entrySet()) { + Assert.assertEquals(entry.getValue(), map.get(entry.getKey())); + } + } + + @Test + public void testIntObjectMap() { + MutableIntObjectMap map = CollectionFactory.newIntObjectMap(); + Assert.assertInstanceOf(IntObjectHashMap.class, map); + + map = CollectionFactory.newIntObjectMap(10); + Assert.assertInstanceOf(IntObjectHashMap.class, map); + int[] keys = Whitebox.getInternalState(map, "keys"); + Assert.assertEquals(32, keys.length); + Object[] values = Whitebox.getInternalState(map, "values"); + Assert.assertEquals(32, values.length); + + map = CollectionFactory.newIntObjectMap(1, "A", 2, "B"); + map = CollectionFactory.newIntObjectMap(map); + Assert.assertInstanceOf(IntObjectHashMap.class, map); + Assert.assertEquals(2, map.size()); + Assert.assertEquals("A", map.get(1)); + Assert.assertEquals("B", map.get(2)); + } + + @Test + public void testIdSet() { + factory = new CollectionFactory(CollectionType.EC); + IdSet idSet = factory.newIdSet(); + Set ids = Whitebox.getInternalState(idSet, "nonNumberIds"); + Assert.assertInstanceOf(UnifiedSet.class, ids); + Assert.assertInstanceOf(LongHashSet.class, + Whitebox.getInternalState(idSet, "numberIds")); + + factory = new CollectionFactory(CollectionType.FU); + idSet = factory.newIdSet(); + ids = Whitebox.getInternalState(idSet, "nonNumberIds"); + Assert.assertInstanceOf(ObjectOpenHashSet.class, ids); + Assert.assertInstanceOf(LongHashSet.class, + Whitebox.getInternalState(idSet, "numberIds")); + + factory = new CollectionFactory(CollectionType.JCF); + idSet = factory.newIdSet(); + ids = Whitebox.getInternalState(idSet, "nonNumberIds"); + Assert.assertInstanceOf(HashSet.class, ids); + Assert.assertInstanceOf(LongHashSet.class, + Whitebox.getInternalState(idSet, "numberIds")); + + idSet = CollectionFactory.newIdSet(CollectionType.EC); + ids = Whitebox.getInternalState(idSet, "nonNumberIds"); + Assert.assertInstanceOf(UnifiedSet.class, ids); + Assert.assertInstanceOf(LongHashSet.class, + Whitebox.getInternalState(idSet, "numberIds")); + + idSet = CollectionFactory.newIdSet(CollectionType.FU); + ids = Whitebox.getInternalState(idSet, "nonNumberIds"); + Assert.assertInstanceOf(ObjectOpenHashSet.class, ids); + Assert.assertInstanceOf(LongHashSet.class, + Whitebox.getInternalState(idSet, "numberIds")); + + idSet = CollectionFactory.newIdSet(CollectionType.JCF); + ids = Whitebox.getInternalState(idSet, "nonNumberIds"); + Assert.assertInstanceOf(HashSet.class, ids); + Assert.assertInstanceOf(LongHashSet.class, + Whitebox.getInternalState(idSet, "numberIds")); + } +} diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/JsonUtilTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/JsonUtilTest.java index eb2521e072..b3ccc2066b 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/JsonUtilTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/util/JsonUtilTest.java @@ -20,11 +20,12 @@ package com.baidu.hugegraph.unit.util; import java.util.Arrays; +import java.util.Date; import java.util.List; -import java.util.Map; import java.util.UUID; import org.apache.tinkerpop.shaded.jackson.core.type.TypeReference; +import org.eclipse.collections.api.map.primitive.MutableIntObjectMap; import org.junit.Test; import org.mockito.Mockito; @@ -53,8 +54,8 @@ import com.baidu.hugegraph.unit.BaseUnitTest; import com.baidu.hugegraph.unit.FakeObjects; import com.baidu.hugegraph.util.JsonUtil; +import com.baidu.hugegraph.util.collection.CollectionFactory; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; public class JsonUtilTest extends BaseUnitTest { @@ -234,7 +235,8 @@ public void testSerializeVertexWithNumberId() { Id id = IdGenerator.of(123456L); HugeVertex vertex = new HugeVertex(fakeObject.graph(), id, vl); - Map> properties = ImmutableMap.of( + MutableIntObjectMap> properties = + CollectionFactory.newIntObjectMap( name.id(), new HugeVertexProperty<>(vertex, name, "marko"), age.id(), new HugeVertexProperty<>(vertex, age, 29), city.id(), new HugeVertexProperty<>(vertex, city, "Beijing") @@ -282,9 +284,10 @@ public void testSerializeEdge() { Whitebox.setInternalState(edge, "sourceVertex", source); Whitebox.setInternalState(edge, "targetVertex", target); - Map> properties = ImmutableMap.of( - date.id(), new HugeEdgeProperty<>(edge, date, - Utils.date("2019-03-12")), + Date dateValue = Utils.date("2019-03-12"); + MutableIntObjectMap> properties = + CollectionFactory.newIntObjectMap( + date.id(), new HugeEdgeProperty<>(edge, date, dateValue), weight.id(), new HugeEdgeProperty<>(edge, weight, 0.8) ); Whitebox.setInternalState(edge, "properties", properties);