From 20d1e5228e39bd974079891fc713574fb14798c5 Mon Sep 17 00:00:00 2001 From: Jermy Li Date: Mon, 4 Dec 2023 12:43:32 +0800 Subject: [PATCH 1/9] chore: move server info into GlobalMasterInfo (#2370) * chore: move server info into GlobalMasterInfo Change-Id: Id854892333115fd45d7c8b9799d255627541b2ad * fix testClearAndInit() and export GlobalMasterInfo Change-Id: Ic2878c5359e7f55fcd11986aa0bf79241c7ee9ab * enhence ServerInfoManager.heartbeat() Change-Id: I6932893c4be8331547f3b721083ca00430f85e58 * move RoleElectionStateMachineTest to UnitTestSuite Change-Id: I7b3e8e2867dcf1063a726c8005f1c34dbd218f7c --- .../hugegraph/api/filter/RedirectFilter.java | 12 +- .../hugegraph/auth/HugeGraphAuthProxy.java | 10 +- .../apache/hugegraph/core/GraphManager.java | 94 +++++------ .../java/org/apache/hugegraph/HugeGraph.java | 8 +- .../apache/hugegraph/StandardHugeGraph.java | 44 ++--- .../job/computer/AbstractComputer.java | 8 +- .../masterelection/GlobalMasterInfo.java | 89 ++++++++--- .../RoleElectionStateMachine.java | 2 +- ...MachineCallback.java => RoleListener.java} | 2 +- .../StandardRoleElectionStateMachine.java | 53 +++--- ...allback.java => StandardRoleListener.java} | 63 ++++---- .../masterelection/StateMachineContext.java | 2 +- .../hugegraph/task/ServerInfoManager.java | 151 +++++++++++------- .../hugegraph/task/StandardTaskScheduler.java | 49 +++--- .../apache/hugegraph/task/TaskManager.java | 58 +++---- .../assembly/static/conf/gremlin-server.yaml | 3 +- .../travis/conf-raft1/gremlin-server.yaml | 6 +- .../travis/conf-raft2/gremlin-server.yaml | 6 +- .../travis/conf-raft3/gremlin-server.yaml | 6 +- .../apache/hugegraph/example/ExampleUtil.java | 7 +- .../apache/hugegraph/api/GremlinApiTest.java | 2 +- .../apache/hugegraph/core/BaseCoreTest.java | 5 +- .../apache/hugegraph/core/CoreTestSuite.java | 4 +- .../hugegraph/core/MultiGraphsTest.java | 11 +- .../core/RoleElectionStateMachineTest.java | 54 ++++--- .../apache/hugegraph/tinkerpop/TestGraph.java | 22 ++- .../apache/hugegraph/unit/UnitTestSuite.java | 2 + .../unit/core/SecurityManagerTest.java | 13 +- 28 files changed, 438 insertions(+), 348 deletions(-) rename hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/{StateMachineCallback.java => RoleListener.java} (96%) rename hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/{StandardStateMachineCallback.java => StandardRoleListener.java} (67%) diff --git a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/filter/RedirectFilter.java b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/filter/RedirectFilter.java index e675dd9554..3fdfd7689c 100644 --- a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/filter/RedirectFilter.java +++ b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/filter/RedirectFilter.java @@ -82,16 +82,16 @@ public void filter(ContainerRequestContext context) throws IOException { return; } - GlobalMasterInfo globalMasterInfo = manager.globalMasterInfo(); - if (globalMasterInfo == null || !globalMasterInfo.isFeatureSupport()) { + GlobalMasterInfo globalNodeInfo = manager.globalNodeRoleInfo(); + if (globalNodeInfo == null || !globalNodeInfo.supportElection()) { return; } - GlobalMasterInfo.NodeInfo masterNodeInfo = globalMasterInfo.nodeInfo(); - if (masterNodeInfo == null || masterNodeInfo.isMaster() || - StringUtils.isEmpty(masterNodeInfo.url())) { + GlobalMasterInfo.NodeInfo masterInfo = globalNodeInfo.masterInfo(); + if (masterInfo == null || masterInfo.isMaster() || + StringUtils.isEmpty(masterInfo.nodeUrl())) { return; } - String url = masterNodeInfo.url(); + String url = masterInfo.nodeUrl(); URI redirectUri; try { diff --git a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/auth/HugeGraphAuthProxy.java b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/auth/HugeGraphAuthProxy.java index 2435e2667a..d22b86bab4 100644 --- a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/auth/HugeGraphAuthProxy.java +++ b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/auth/HugeGraphAuthProxy.java @@ -55,6 +55,7 @@ import org.apache.hugegraph.exception.NotSupportException; import org.apache.hugegraph.iterator.FilterIterator; import org.apache.hugegraph.iterator.MapperIterator; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.masterelection.RoleElectionStateMachine; import org.apache.hugegraph.rpc.RpcServiceConfig4Client; import org.apache.hugegraph.rpc.RpcServiceConfig4Server; @@ -78,7 +79,6 @@ import org.apache.hugegraph.type.Nameable; import org.apache.hugegraph.type.define.GraphMode; import org.apache.hugegraph.type.define.GraphReadMode; -import org.apache.hugegraph.type.define.NodeRole; import org.apache.hugegraph.util.E; import org.apache.hugegraph.util.Log; import org.apache.hugegraph.util.RateLimiter; @@ -669,9 +669,9 @@ public void waitReady(RpcServer rpcServer) { } @Override - public void serverStarted(Id serverId, NodeRole serverRole) { + public void serverStarted(GlobalMasterInfo nodeInfo) { this.verifyAdminPermission(); - this.hugegraph.serverStarted(serverId, serverRole); + this.hugegraph.serverStarted(nodeInfo); } @Override @@ -776,9 +776,9 @@ public void resumeSnapshot() { } @Override - public void create(String configPath, Id server, NodeRole role) { + public void create(String configPath, GlobalMasterInfo nodeInfo) { this.verifyPermission(HugePermission.WRITE, ResourceType.STATUS); - this.hugegraph.create(configPath, server, role); + this.hugegraph.create(configPath, nodeInfo); } @Override diff --git a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/core/GraphManager.java b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/core/GraphManager.java index 2c73b5ee93..b8770ca7df 100644 --- a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/core/GraphManager.java +++ b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/core/GraphManager.java @@ -23,8 +23,6 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -41,7 +39,6 @@ import org.apache.hugegraph.backend.BackendException; import org.apache.hugegraph.backend.cache.Cache; import org.apache.hugegraph.backend.cache.CacheManager; -import org.apache.hugegraph.backend.id.Id; import org.apache.hugegraph.backend.id.IdGenerator; import org.apache.hugegraph.backend.store.BackendStoreInfo; import org.apache.hugegraph.config.CoreOptions; @@ -53,7 +50,7 @@ import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.masterelection.RoleElectionOptions; import org.apache.hugegraph.masterelection.RoleElectionStateMachine; -import org.apache.hugegraph.masterelection.StandardStateMachineCallback; +import org.apache.hugegraph.masterelection.StandardRoleListener; import org.apache.hugegraph.metrics.MetricsUtil; import org.apache.hugegraph.metrics.ServerReporter; import org.apache.hugegraph.rpc.RpcClientProvider; @@ -88,14 +85,11 @@ public final class GraphManager { private final HugeAuthenticator authenticator; private final RpcServer rpcServer; private final RpcClientProvider rpcClient; - private final HugeConfig conf; - - private RoleElectionStateMachine roleStateWorker; - private GlobalMasterInfo globalMasterInfo; - private Id server; - private NodeRole role; + private RoleElectionStateMachine roleStateMachine; + private GlobalMasterInfo globalNodeRoleInfo; + private final HugeConfig conf; private final EventHub eventHub; public GraphManager(HugeConfig conf, EventHub hub) { @@ -104,6 +98,10 @@ public GraphManager(HugeConfig conf, EventHub hub) { this.authenticator = HugeAuthenticator.loadAuthenticator(conf); this.rpcServer = new RpcServer(conf); this.rpcClient = new RpcClientProvider(conf); + + this.roleStateMachine = null; + this.globalNodeRoleInfo = new GlobalMasterInfo(); + this.eventHub = hub; this.conf = conf; } @@ -141,8 +139,7 @@ public void loadGraphs(Map graphConfs) { } } - public HugeGraph cloneGraph(String name, String newName, - String configText) { + public HugeGraph cloneGraph(String name, String newName, String configText) { /* * 0. check and modify params * 1. create graph instance @@ -270,6 +267,10 @@ public AuthManager authManager() { return this.authenticator().authManager(); } + public GlobalMasterInfo globalNodeRoleInfo() { + return this.globalNodeRoleInfo; + } + public void close() { for (Graph graph : this.graphs.values()) { try { @@ -280,8 +281,8 @@ public void close() { } this.destroyRpcServer(); this.unlistenChanges(); - if (this.roleStateWorker != null) { - this.roleStateWorker.shutdown(); + if (this.roleStateMachine != null) { + this.roleStateMachine.shutdown(); } } @@ -414,8 +415,7 @@ private void waitGraphsReady() { LOG.info("RpcServer is not enabled, skip wait graphs ready"); return; } - com.alipay.remoting.rpc.RpcServer remotingRpcServer = - this.remotingRpcServer(); + com.alipay.remoting.rpc.RpcServer remotingRpcServer = this.remotingRpcServer(); for (String graphName : this.graphs.keySet()) { HugeGraph graph = this.graph(graphName); graph.waitReady(remotingRpcServer); @@ -433,7 +433,7 @@ private void checkBackendVersionOrExit(HugeConfig config) { if (this.requireAuthentication()) { String token = config.get(ServerOptions.AUTH_ADMIN_TOKEN); try { - this.authenticator.initAdminUser(token); + this.authenticator().initAdminUser(token); } catch (Exception e) { throw new BackendException( "The backend store of '%s' can't " + @@ -455,65 +455,57 @@ private void checkBackendVersionOrExit(HugeConfig config) { } private void serverStarted(HugeConfig config) { - String server = config.get(ServerOptions.SERVER_ID); + String id = config.get(ServerOptions.SERVER_ID); String role = config.get(ServerOptions.SERVER_ROLE); - E.checkArgument(StringUtils.isNotEmpty(server), + E.checkArgument(StringUtils.isNotEmpty(id), "The server name can't be null or empty"); E.checkArgument(StringUtils.isNotEmpty(role), "The server role can't be null or empty"); - this.server = IdGenerator.of(server); - this.role = NodeRole.valueOf(role.toUpperCase()); - boolean supportRoleStateWorker = this.supportRoleStateWorker(); - if (supportRoleStateWorker) { - this.role = NodeRole.WORKER; + NodeRole nodeRole = NodeRole.valueOf(role.toUpperCase()); + boolean supportRoleElection = !nodeRole.computer() && + this.supportRoleElection(); + if (supportRoleElection) { + // Init any server as Worker role, then do role election + nodeRole = NodeRole.WORKER; } + this.globalNodeRoleInfo.initNodeId(IdGenerator.of(id)); + this.globalNodeRoleInfo.initNodeRole(nodeRole); + for (String graph : this.graphs()) { HugeGraph hugegraph = this.graph(graph); assert hugegraph != null; - hugegraph.serverStarted(this.server, this.role); + hugegraph.serverStarted(this.globalNodeRoleInfo); } - if (supportRoleStateWorker) { - this.initRoleStateWorker(); + if (supportRoleElection) { + this.initRoleStateMachine(); } } - private void initRoleStateWorker() { - E.checkArgument(this.roleStateWorker == null, "Repetition init"); - Executor applyThread = Executors.newSingleThreadExecutor(); - this.roleStateWorker = this.authenticator().graph().roleElectionStateMachine(); - this.globalMasterInfo = new GlobalMasterInfo(); - StandardStateMachineCallback stateMachineCallback = new StandardStateMachineCallback( - TaskManager.instance(), - this.globalMasterInfo); - applyThread.execute(() -> { - this.roleStateWorker.apply(stateMachineCallback); - }); - } - - public GlobalMasterInfo globalMasterInfo() { - return this.globalMasterInfo; + private void initRoleStateMachine() { + E.checkArgument(this.roleStateMachine == null, + "Repeated initialization of role state worker"); + this.globalNodeRoleInfo.supportElection(true); + this.roleStateMachine = this.authenticator().graph().roleElectionStateMachine(); + StandardRoleListener listener = new StandardRoleListener(TaskManager.instance(), + this.globalNodeRoleInfo); + this.roleStateMachine.start(listener); } - private boolean supportRoleStateWorker() { - if (this.role.computer()) { - return false; - } - + private boolean supportRoleElection() { try { if (!(this.authenticator() instanceof StandardAuthenticator)) { LOG.info("{} authenticator does not support role election currently", this.authenticator().getClass().getSimpleName()); return false; } + return true; } catch (IllegalStateException e) { - LOG.info("Unconfigured StandardAuthenticator, not support role election currently"); + LOG.info("{}, does not support role election currently", e.getMessage()); return false; } - - return true; } private void addMetrics(HugeConfig config) { @@ -591,7 +583,7 @@ private HugeGraph createGraph(HugeConfig config, String name) { graph = (HugeGraph) GraphFactory.open(config); // Init graph and start it - graph.create(this.graphsDir, this.server, this.role); + graph.create(this.graphsDir, this.globalNodeRoleInfo); } catch (Throwable e) { LOG.error("Failed to create graph '{}' due to: {}", name, e.getMessage(), e); diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/HugeGraph.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/HugeGraph.java index cd287c47be..85c093e59e 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/HugeGraph.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/HugeGraph.java @@ -30,6 +30,7 @@ import org.apache.hugegraph.backend.store.raft.RaftGroupManager; import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.config.TypedOption; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.masterelection.RoleElectionStateMachine; import org.apache.hugegraph.rpc.RpcServiceConfig4Client; import org.apache.hugegraph.rpc.RpcServiceConfig4Server; @@ -44,12 +45,11 @@ import org.apache.hugegraph.task.TaskScheduler; import org.apache.hugegraph.traversal.optimize.HugeCountStepStrategy; import org.apache.hugegraph.traversal.optimize.HugeGraphStepStrategy; -import org.apache.hugegraph.traversal.optimize.HugeVertexStepStrategy; import org.apache.hugegraph.traversal.optimize.HugePrimaryKeyStrategy; +import org.apache.hugegraph.traversal.optimize.HugeVertexStepStrategy; import org.apache.hugegraph.type.HugeType; import org.apache.hugegraph.type.define.GraphMode; import org.apache.hugegraph.type.define.GraphReadMode; -import org.apache.hugegraph.type.define.NodeRole; import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategies; import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Graph; @@ -201,7 +201,7 @@ public interface HugeGraph extends Graph { void waitReady(RpcServer rpcServer); - void serverStarted(Id serverId, NodeRole serverRole); + void serverStarted(GlobalMasterInfo nodeInfo); boolean started(); @@ -221,7 +221,7 @@ public interface HugeGraph extends Graph { void resumeSnapshot(); - void create(String configPath, Id server, NodeRole role); + void create(String configPath, GlobalMasterInfo nodeInfo); void drop(); diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/StandardHugeGraph.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/StandardHugeGraph.java index db37d0a4bd..c671056e04 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/StandardHugeGraph.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/StandardHugeGraph.java @@ -51,7 +51,6 @@ import org.apache.hugegraph.backend.store.raft.RaftBackendStoreProvider; import org.apache.hugegraph.backend.store.raft.RaftGroupManager; import org.apache.hugegraph.backend.store.ram.RamTable; -import org.apache.hugegraph.task.EphemeralJobQueue; import org.apache.hugegraph.backend.tx.GraphTransaction; import org.apache.hugegraph.backend.tx.SchemaTransaction; import org.apache.hugegraph.config.CoreOptions; @@ -64,6 +63,7 @@ import org.apache.hugegraph.job.EphemeralJob; import org.apache.hugegraph.masterelection.ClusterRoleStore; import org.apache.hugegraph.masterelection.Config; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.masterelection.RoleElectionConfig; import org.apache.hugegraph.masterelection.RoleElectionOptions; import org.apache.hugegraph.masterelection.RoleElectionStateMachine; @@ -84,13 +84,13 @@ import org.apache.hugegraph.structure.HugeFeatures; import org.apache.hugegraph.structure.HugeVertex; import org.apache.hugegraph.structure.HugeVertexProperty; +import org.apache.hugegraph.task.EphemeralJobQueue; import org.apache.hugegraph.task.ServerInfoManager; import org.apache.hugegraph.task.TaskManager; import org.apache.hugegraph.task.TaskScheduler; import org.apache.hugegraph.type.HugeType; import org.apache.hugegraph.type.define.GraphMode; import org.apache.hugegraph.type.define.GraphReadMode; -import org.apache.hugegraph.type.define.NodeRole; import org.apache.hugegraph.util.ConfigUtil; import org.apache.hugegraph.util.DateUtil; import org.apache.hugegraph.util.E; @@ -267,15 +267,15 @@ public BackendFeatures backendStoreFeatures() { } @Override - public void serverStarted(Id serverId, NodeRole serverRole) { + public void serverStarted(GlobalMasterInfo nodeInfo) { LOG.info("Init system info for graph '{}'", this.name); this.initSystemInfo(); LOG.info("Init server info [{}-{}] for graph '{}'...", - serverId, serverRole, this.name); - this.serverInfoManager().initServerInfo(serverId, serverRole); + nodeInfo.nodeId(), nodeInfo.nodeRole(), this.name); + this.serverInfoManager().initServerInfo(nodeInfo); - this.initRoleStateWorker(serverId); + this.initRoleStateMachine(nodeInfo.nodeId()); // TODO: check necessary? LOG.info("Check olap property-key tables for graph '{}'", this.name); @@ -291,16 +291,18 @@ public void serverStarted(Id serverId, NodeRole serverRole) { this.started = true; } - private void initRoleStateWorker(Id serverId) { - Config roleStateMachineConfig = new RoleElectionConfig(serverId.toString(), - this.configuration.get(RoleElectionOptions.NODE_EXTERNAL_URL), - this.configuration.get(RoleElectionOptions.EXCEEDS_FAIL_COUNT), - this.configuration.get(RoleElectionOptions.RANDOM_TIMEOUT_MILLISECOND), - this.configuration.get(RoleElectionOptions.HEARTBEAT_INTERVAL_SECOND), - this.configuration.get(RoleElectionOptions.MASTER_DEAD_TIMES), - this.configuration.get(RoleElectionOptions.BASE_TIMEOUT_MILLISECOND)); - ClusterRoleStore clusterRoleStore = new StandardClusterRoleStore(this.params); - this.roleElectionStateMachine = new StandardRoleElectionStateMachine(roleStateMachineConfig, clusterRoleStore); + private void initRoleStateMachine(Id serverId) { + HugeConfig conf = this.configuration; + Config roleConfig = new RoleElectionConfig(serverId.toString(), + conf.get(RoleElectionOptions.NODE_EXTERNAL_URL), + conf.get(RoleElectionOptions.EXCEEDS_FAIL_COUNT), + conf.get(RoleElectionOptions.RANDOM_TIMEOUT_MILLISECOND), + conf.get(RoleElectionOptions.HEARTBEAT_INTERVAL_SECOND), + conf.get(RoleElectionOptions.MASTER_DEAD_TIMES), + conf.get(RoleElectionOptions.BASE_TIMEOUT_MILLISECOND)); + ClusterRoleStore roleStore = new StandardClusterRoleStore(this.params); + this.roleElectionStateMachine = new StandardRoleElectionStateMachine(roleConfig, + roleStore); } @Override @@ -399,8 +401,7 @@ public void truncateBackend() { try { this.storeProvider.truncate(); // TODO: remove this after serverinfo saved in etcd - this.serverStarted(this.serverInfoManager().selfServerId(), - this.serverInfoManager().selfServerRole()); + this.serverStarted(this.serverInfoManager().globalNodeRoleInfo()); } finally { LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); } @@ -974,9 +975,9 @@ public synchronized void close() throws Exception { } @Override - public void create(String configPath, Id server, NodeRole role) { + public void create(String configPath, GlobalMasterInfo nodeInfo) { this.initBackend(); - this.serverStarted(server, role); + this.serverStarted(nodeInfo); // Write config to disk file String confPath = ConfigUtil.writeToFile(configPath, this.name(), @@ -1052,8 +1053,7 @@ public TaskScheduler taskScheduler() { } private ServerInfoManager serverInfoManager() { - ServerInfoManager manager = this.taskManager - .getServerInfoManager(this.params); + ServerInfoManager manager = this.taskManager.getServerInfoManager(this.params); E.checkState(manager != null, "Can't find server info manager for graph '%s'", this); return manager; diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/job/computer/AbstractComputer.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/job/computer/AbstractComputer.java index a40d0001b4..11207ebae5 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/job/computer/AbstractComputer.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/job/computer/AbstractComputer.java @@ -32,16 +32,15 @@ import org.apache.commons.configuration2.tree.ImmutableNode; import org.apache.commons.configuration2.tree.NodeHandler; import org.apache.commons.configuration2.tree.NodeModel; -import org.apache.hugegraph.type.define.Directions; -import org.apache.hugegraph.util.ParameterUtil; -import org.slf4j.Logger; - import org.apache.hugegraph.HugeException; import org.apache.hugegraph.job.ComputerJob; import org.apache.hugegraph.job.Job; import org.apache.hugegraph.traversal.algorithm.HugeTraverser; +import org.apache.hugegraph.type.define.Directions; import org.apache.hugegraph.util.E; import org.apache.hugegraph.util.Log; +import org.apache.hugegraph.util.ParameterUtil; +import org.slf4j.Logger; public abstract class AbstractComputer implements Computer { @@ -85,7 +84,6 @@ public void checkParameters(Map parameters) { @Override public Object call(Job job, Map parameters) { - this.checkAndCollectParameters(parameters); // Read configuration try { diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/GlobalMasterInfo.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/GlobalMasterInfo.java index d7dd127af5..9124fcd0b6 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/GlobalMasterInfo.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/GlobalMasterInfo.java @@ -17,50 +17,103 @@ package org.apache.hugegraph.masterelection; -public class GlobalMasterInfo { +import org.apache.hugegraph.backend.id.Id; +import org.apache.hugegraph.backend.id.IdGenerator; +import org.apache.hugegraph.type.define.NodeRole; +import org.apache.hugegraph.util.E; - private NodeInfo nodeInfo; - private volatile boolean featureSupport; +// TODO: rename to GlobalNodeRoleInfo +public final class GlobalMasterInfo { + + private final static NodeInfo NO_MASTER = new NodeInfo(false, ""); + + private volatile boolean supportElection; + private volatile NodeInfo masterNodeInfo; + + private volatile Id nodeId; + private volatile NodeRole nodeRole; public GlobalMasterInfo() { - this.featureSupport = false; - this.nodeInfo = new NodeInfo(false, ""); + this(NO_MASTER); + } + + public GlobalMasterInfo(NodeInfo masterInfo) { + this.supportElection = false; + this.masterNodeInfo = masterInfo; + + this.nodeId = null; + this.nodeRole = null; + } + + public void supportElection(boolean featureSupport) { + this.supportElection = featureSupport; + } + + public boolean supportElection() { + return this.supportElection; } - public final void nodeInfo(boolean isMaster, String url) { + public void resetMasterInfo() { + this.masterNodeInfo = NO_MASTER; + } + + public void masterInfo(boolean isMaster, String nodeUrl) { // final can avoid instruction rearrangement, visibility can be ignored - final NodeInfo tmp = new NodeInfo(isMaster, url); - this.nodeInfo = tmp; + this.masterNodeInfo = new NodeInfo(isMaster, nodeUrl); + } + + public NodeInfo masterInfo() { + return this.masterNodeInfo; + } + + public Id nodeId() { + return this.nodeId; + } + + public NodeRole nodeRole() { + return this.nodeRole; + } + + public void initNodeId(Id id) { + this.nodeId = id; } - public final NodeInfo nodeInfo() { - return this.nodeInfo; + public void initNodeRole(NodeRole role) { + E.checkArgument(role != null, "The server role can't be null"); + E.checkArgument(this.nodeRole == null, + "The server role can't be init twice"); + this.nodeRole = role; } - public void isFeatureSupport(boolean featureSupport) { - this.featureSupport = featureSupport; + public void changeNodeRole(NodeRole role) { + E.checkArgument(role != null, "The server role can't be null"); + this.nodeRole = role; } - public boolean isFeatureSupport() { - return this.featureSupport; + public static GlobalMasterInfo master(String nodeId) { + NodeInfo masterInfo = new NodeInfo(true, nodeId); + GlobalMasterInfo nodeInfo = new GlobalMasterInfo(masterInfo); + nodeInfo.nodeId = IdGenerator.of(nodeId); + nodeInfo.nodeRole = NodeRole.MASTER; + return nodeInfo; } public static class NodeInfo { private final boolean isMaster; - private final String url; + private final String nodeUrl; public NodeInfo(boolean isMaster, String url) { this.isMaster = isMaster; - this.url = url; + this.nodeUrl = url; } public boolean isMaster() { return this.isMaster; } - public String url() { - return this.url; + public String nodeUrl() { + return this.nodeUrl; } } } diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/RoleElectionStateMachine.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/RoleElectionStateMachine.java index 920bc104f1..2a33d1bf6c 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/RoleElectionStateMachine.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/RoleElectionStateMachine.java @@ -21,5 +21,5 @@ public interface RoleElectionStateMachine { void shutdown(); - void apply(StateMachineCallback stateMachineCallback); + void start(RoleListener callback); } diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StateMachineCallback.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/RoleListener.java similarity index 96% rename from hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StateMachineCallback.java rename to hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/RoleListener.java index 35abbe3402..e99db4c16f 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StateMachineCallback.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/RoleListener.java @@ -17,7 +17,7 @@ package org.apache.hugegraph.masterelection; -public interface StateMachineCallback { +public interface RoleListener { void onAsRoleMaster(StateMachineContext context); diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardRoleElectionStateMachine.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardRoleElectionStateMachine.java index aa284def60..a0e2601d49 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardRoleElectionStateMachine.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardRoleElectionStateMachine.java @@ -19,6 +19,8 @@ import java.security.SecureRandom; import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.locks.LockSupport; import org.apache.hugegraph.util.E; @@ -29,25 +31,36 @@ public class StandardRoleElectionStateMachine implements RoleElectionStateMachin private static final Logger LOG = Log.logger(StandardRoleElectionStateMachine.class); - private volatile boolean shutdown; private final Config config; + private final ClusterRoleStore roleStore; + private final ExecutorService applyThread; + + private volatile boolean shutdown; private volatile RoleState state; - private final ClusterRoleStore clusterRoleStore; - public StandardRoleElectionStateMachine(Config config, ClusterRoleStore clusterRoleStore) { + public StandardRoleElectionStateMachine(Config config, ClusterRoleStore roleStore) { this.config = config; - this.clusterRoleStore = clusterRoleStore; + this.roleStore = roleStore; + this.applyThread = Executors.newSingleThreadExecutor(); this.state = new UnknownState(null); this.shutdown = false; } @Override public void shutdown() { + if (this.shutdown) { + return; + } this.shutdown = true; + this.applyThread.shutdown(); } @Override - public void apply(StateMachineCallback stateMachineCallback) { + public void start(RoleListener stateMachineCallback) { + this.applyThread.execute(() -> this.apply(stateMachineCallback)); + } + + private void apply(RoleListener stateMachineCallback) { int failCount = 0; StateMachineContextImpl context = new StateMachineContextImpl(this); while (!this.shutdown) { @@ -73,13 +86,17 @@ public void apply(StateMachineCallback stateMachineCallback) { } } + protected ClusterRoleStore roleStore() { + return this.roleStore; + } + private interface RoleState { SecureRandom SECURE_RANDOM = new SecureRandom(); RoleState transform(StateMachineContext context); - Callback callback(StateMachineCallback callback); + Callback callback(RoleListener callback); static void heartBeatPark(StateMachineContext context) { long heartBeatIntervalSecond = context.config().heartBeatIntervalSecond(); @@ -110,7 +127,7 @@ public UnknownState(Integer epoch) { @Override public RoleState transform(StateMachineContext context) { - ClusterRoleStore adapter = context.adapter(); + ClusterRoleStore adapter = context.roleStore(); Optional clusterRoleOpt = adapter.query(); if (!clusterRoleOpt.isPresent()) { context.reset(); @@ -137,7 +154,7 @@ public RoleState transform(StateMachineContext context) { } @Override - public Callback callback(StateMachineCallback callback) { + public Callback callback(RoleListener callback) { return callback::unknown; } } @@ -158,7 +175,7 @@ public RoleState transform(StateMachineContext context) { } @Override - public Callback callback(StateMachineCallback callback) { + public Callback callback(RoleListener callback) { return callback::onAsRoleAbdication; } } @@ -175,7 +192,7 @@ public MasterState(ClusterRole clusterRole) { public RoleState transform(StateMachineContext context) { this.clusterRole.increaseClock(); RoleState.heartBeatPark(context); - if (context.adapter().updateIfNodePresent(this.clusterRole)) { + if (context.roleStore().updateIfNodePresent(this.clusterRole)) { return this; } context.reset(); @@ -184,7 +201,7 @@ public RoleState transform(StateMachineContext context) { } @Override - public Callback callback(StateMachineCallback callback) { + public Callback callback(RoleListener callback) { return callback::onAsRoleMaster; } } @@ -216,7 +233,7 @@ public RoleState transform(StateMachineContext context) { } @Override - public Callback callback(StateMachineCallback callback) { + public Callback callback(RoleListener callback) { return callback::onAsRoleWorker; } @@ -255,7 +272,7 @@ public RoleState transform(StateMachineContext context) { context.config().url(), epoch); // The master failover completed context.epoch(clusterRole.epoch()); - if (context.adapter().updateIfNodePresent(clusterRole)) { + if (context.roleStore().updateIfNodePresent(clusterRole)) { context.master(new MasterServerInfoImpl(clusterRole.node(), clusterRole.url())); return new MasterState(clusterRole); } else { @@ -264,7 +281,7 @@ public RoleState transform(StateMachineContext context) { } @Override - public Callback callback(StateMachineCallback callback) { + public Callback callback(RoleListener callback) { return callback::onAsRoleCandidate; } } @@ -303,8 +320,8 @@ public void epoch(Integer epoch) { } @Override - public ClusterRoleStore adapter() { - return this.machine.adapter(); + public ClusterRoleStore roleStore() { + return this.machine.roleStore(); } @Override @@ -348,8 +365,4 @@ public String node() { return this.node; } } - - protected ClusterRoleStore adapter() { - return this.clusterRoleStore; - } } diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardStateMachineCallback.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardRoleListener.java similarity index 67% rename from hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardStateMachineCallback.java rename to hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardRoleListener.java index 28e01d2913..f2bb94f521 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardStateMachineCallback.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StandardRoleListener.java @@ -23,84 +23,83 @@ import org.apache.hugegraph.util.Log; import org.slf4j.Logger; -public class StandardStateMachineCallback implements StateMachineCallback { +public class StandardRoleListener implements RoleListener { - private static final Logger LOG = Log.logger(StandardStateMachineCallback.class); + private static final Logger LOG = Log.logger(StandardRoleListener.class); private final TaskManager taskManager; - private final GlobalMasterInfo globalMasterInfo; + private final GlobalMasterInfo roleInfo; - private boolean isMaster = false; + private volatile boolean selfIsMaster; - public StandardStateMachineCallback(TaskManager taskManager, - GlobalMasterInfo globalMasterInfo) { + public StandardRoleListener(TaskManager taskManager, + GlobalMasterInfo roleInfo) { this.taskManager = taskManager; - this.taskManager.enableRoleElected(true); - this.globalMasterInfo = globalMasterInfo; - this.globalMasterInfo.isFeatureSupport(true); + this.taskManager.enableRoleElection(); + this.roleInfo = roleInfo; + this.selfIsMaster = false; } @Override public void onAsRoleMaster(StateMachineContext context) { - if (!isMaster) { + if (!selfIsMaster) { this.taskManager.onAsRoleMaster(); LOG.info("Server {} change to master role", context.config().node()); } - this.initGlobalMasterInfo(context); - this.isMaster = true; + this.updateMasterInfo(context); + this.selfIsMaster = true; } @Override public void onAsRoleWorker(StateMachineContext context) { - if (isMaster) { + if (this.selfIsMaster) { this.taskManager.onAsRoleWorker(); LOG.info("Server {} change to worker role", context.config().node()); } - this.initGlobalMasterInfo(context); - this.isMaster = false; + this.updateMasterInfo(context); + this.selfIsMaster = false; } @Override public void onAsRoleCandidate(StateMachineContext context) { + // pass } @Override - public void unknown(StateMachineContext context) { - if (isMaster) { + public void onAsRoleAbdication(StateMachineContext context) { + if (this.selfIsMaster) { this.taskManager.onAsRoleWorker(); LOG.info("Server {} change to worker role", context.config().node()); } - this.initGlobalMasterInfo(context); + this.updateMasterInfo(context); + this.selfIsMaster = false; + } - isMaster = false; + @Override + public void error(StateMachineContext context, Throwable e) { + LOG.error("Server {} exception occurred", context.config().node(), e); } @Override - public void onAsRoleAbdication(StateMachineContext context) { - if (isMaster) { + public void unknown(StateMachineContext context) { + if (this.selfIsMaster) { this.taskManager.onAsRoleWorker(); LOG.info("Server {} change to worker role", context.config().node()); } - this.initGlobalMasterInfo(context); - - isMaster = false; - } + this.updateMasterInfo(context); - @Override - public void error(StateMachineContext context, Throwable e) { - LOG.error("Server {} exception occurred", context.config().node(), e); + this.selfIsMaster = false; } - public void initGlobalMasterInfo(StateMachineContext context) { + public void updateMasterInfo(StateMachineContext context) { StateMachineContext.MasterServerInfo master = context.master(); if (master == null) { - this.globalMasterInfo.nodeInfo(false, ""); + this.roleInfo.resetMasterInfo(); return; } boolean isMaster = Objects.equals(context.node(), master.node()); - String url = master.url(); - this.globalMasterInfo.nodeInfo(isMaster, url); + this.roleInfo.masterInfo(isMaster, master.url()); } } diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StateMachineContext.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StateMachineContext.java index a3eaf89626..587cd417fc 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StateMachineContext.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/masterelection/StateMachineContext.java @@ -33,7 +33,7 @@ public interface StateMachineContext { void master(MasterServerInfo info); - ClusterRoleStore adapter(); + ClusterRoleStore roleStore(); void reset(); diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/ServerInfoManager.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/ServerInfoManager.java index e8cccf88e3..ee14b4ceb2 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/ServerInfoManager.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/ServerInfoManager.java @@ -37,6 +37,7 @@ import org.apache.hugegraph.exception.ConnectionException; import org.apache.hugegraph.iterator.ListIterator; import org.apache.hugegraph.iterator.MapperIterator; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.schema.PropertyKey; import org.apache.hugegraph.schema.VertexLabel; import org.apache.hugegraph.structure.HugeVertex; @@ -61,8 +62,7 @@ public class ServerInfoManager { private final HugeGraphParams graph; private final ExecutorService dbExecutor; - private Id selfServerId; - private NodeRole selfServerRole; + private volatile GlobalMasterInfo globalNodeInfo; private volatile boolean onlySingleNode; private volatile boolean closed; @@ -75,8 +75,7 @@ public ServerInfoManager(HugeGraphParams graph, this.graph = graph; this.dbExecutor = dbExecutor; - this.selfServerId = null; - this.selfServerRole = NodeRole.MASTER; + this.globalNodeInfo = null; this.onlySingleNode = false; this.closed = false; @@ -86,7 +85,7 @@ public void init() { HugeServerInfo.schema(this.graph).initSchemaIfNeeded(); } - public boolean close() { + public synchronized boolean close() { this.closed = true; if (!this.dbExecutor.isShutdown()) { this.removeSelfServerInfo(); @@ -103,40 +102,24 @@ public boolean close() { return true; } - public synchronized void forceInitServerInfo(Id server, NodeRole role) { - if (this.closed) { - return; - } - - E.checkArgument(server != null && role != null, - "The server id or role can't be null"); - this.selfServerId = server; - this.selfServerRole = role; + public synchronized void initServerInfo(GlobalMasterInfo nodeInfo) { + E.checkArgument(nodeInfo != null, "The global node info can't be null"); - this.saveServerInfo(this.selfServerId, this.selfServerRole); - } - - public synchronized void initServerInfo(Id server, NodeRole role) { - E.checkArgument(server != null && role != null, - "The server id or role can't be null"); - this.selfServerId = server; - this.selfServerRole = role; - - HugeServerInfo existed = this.serverInfo(server); + Id serverId = nodeInfo.nodeId(); + HugeServerInfo existed = this.serverInfo(serverId); E.checkArgument(existed == null || !existed.alive(), "The server with name '%s' already in cluster", - server); - if (role.master()) { + serverId); + + if (nodeInfo.nodeRole().master()) { String page = this.supportsPaging() ? PageInfo.PAGE_NONE : null; do { - Iterator servers = this.serverInfos(PAGE_SIZE, - page); + Iterator servers = this.serverInfos(PAGE_SIZE, page); while (servers.hasNext()) { existed = servers.next(); - E.checkArgument(!existed.role().master() || - !existed.alive(), - "Already existed master '%s' in current " + - "cluster", existed.id()); + E.checkArgument(!existed.role().master() || !existed.alive(), + "Already existed master '%s' in current cluster", + existed.id()); } if (page != null) { page = PageInfo.pageInfo(servers); @@ -144,36 +127,80 @@ public synchronized void initServerInfo(Id server, NodeRole role) { } while (page != null); } - // TODO: save ServerInfo at AuthServer - this.saveServerInfo(this.selfServerId, this.selfServerRole); + this.globalNodeInfo = nodeInfo; + + // TODO: save ServerInfo to AuthServer + this.saveServerInfo(this.selfNodeId(), this.selfNodeRole()); + } + + public synchronized void changeServerRole(NodeRole nodeRole) { + if (this.closed) { + return; + } + + this.globalNodeInfo.changeNodeRole(nodeRole); + + // TODO: save ServerInfo to AuthServer + this.saveServerInfo(this.selfNodeId(), this.selfNodeRole()); } - public Id selfServerId() { - return this.selfServerId; + public GlobalMasterInfo globalNodeRoleInfo() { + return this.globalNodeInfo; } - public NodeRole selfServerRole() { - return this.selfServerRole; + public Id selfNodeId() { + if (this.globalNodeInfo == null) { + return null; + } + return this.globalNodeInfo.nodeId(); + } + + public NodeRole selfNodeRole() { + if (this.globalNodeInfo == null) { + return null; + } + return this.globalNodeInfo.nodeRole(); } - public boolean master() { - return this.selfServerRole != null && this.selfServerRole.master(); + public boolean selfIsMaster() { + return this.selfNodeRole() != null && this.selfNodeRole().master(); } public boolean onlySingleNode() { - // Only has one master node + // Only exists one node in the whole master return this.onlySingleNode; } - public void heartbeat() { + public synchronized void heartbeat() { + assert this.graphIsReady(); + HugeServerInfo serverInfo = this.selfServerInfo(); - if (serverInfo == null && this.selfServerId != null && - this.selfServerRole != NodeRole.MASTER) { - serverInfo = this.saveServerInfo(this.selfServerId, - this.selfServerRole); + if (serverInfo != null) { + // Update heartbeat time for this server + serverInfo.updateTime(DateUtil.now()); + this.save(serverInfo); + return; } - serverInfo.updateTime(DateUtil.now()); - this.save(serverInfo); + + /* ServerInfo is missing */ + if (this.selfNodeId() == null) { + // Ignore if ServerInfo is not initialized + LOG.info("ServerInfo is missing: {}, may not be initialized yet"); + return; + } + if (this.selfIsMaster()) { + // On master node, just wait for ServerInfo re-init + LOG.warn("ServerInfo is missing: {}, may be cleared before", + this.selfNodeId()); + return; + } + /* + * Missing server info on non-master node, may be caused by graph + * truncated on master node then synced by raft. + * TODO: we just patch it here currently, to be improved. + */ + serverInfo = this.saveServerInfo(this.selfNodeId(), this.selfNodeRole()); + assert serverInfo != null; } public synchronized void decreaseLoad(int load) { @@ -188,7 +215,7 @@ public int calcMaxLoad() { return 10000; } - protected boolean graphReady() { + protected boolean graphIsReady() { return !this.closed && this.graph.started() && this.graph.initialized(); } @@ -242,8 +269,8 @@ private GraphTransaction tx() { return this.graph.systemTransaction(); } - private HugeServerInfo saveServerInfo(Id server, NodeRole role) { - HugeServerInfo serverInfo = new HugeServerInfo(server, role); + private HugeServerInfo saveServerInfo(Id serverId, NodeRole serverRole) { + HugeServerInfo serverInfo = new HugeServerInfo(serverId, serverRole); serverInfo.maxLoad(this.calcMaxLoad()); this.save(serverInfo); @@ -310,16 +337,16 @@ private V call(Callable callable) { } private HugeServerInfo selfServerInfo() { - HugeServerInfo selfServerInfo = this.serverInfo(this.selfServerId); - if (selfServerInfo == null) { - LOG.warn("ServerInfo is missing: {}", this.selfServerId); + HugeServerInfo selfServerInfo = this.serverInfo(this.selfNodeId()); + if (selfServerInfo == null && this.selfNodeId() != null) { + LOG.warn("ServerInfo is missing: {}", this.selfNodeId()); } return selfServerInfo; } - private HugeServerInfo serverInfo(Id server) { + private HugeServerInfo serverInfo(Id serverId) { return this.call(() -> { - Iterator vertices = this.tx().queryVertices(server); + Iterator vertices = this.tx().queryVertices(serverId); Vertex vertex = QueryResults.one(vertices); if (vertex == null) { return null; @@ -335,19 +362,19 @@ private HugeServerInfo removeSelfServerInfo() { * backend store, initServerInfo() is not called in this case, so * this.selfServerId is null at this time. */ - if (this.selfServerId != null && this.graph.initialized()) { - return this.removeServerInfo(this.selfServerId); + if (this.selfNodeId() != null && this.graph.initialized()) { + return this.removeServerInfo(this.selfNodeId()); } return null; } - private HugeServerInfo removeServerInfo(Id server) { - if (server == null) { + private HugeServerInfo removeServerInfo(Id serverId) { + if (serverId == null) { return null; } - LOG.info("Remove server info: {}", server); + LOG.info("Remove server info: {}", serverId); return this.call(() -> { - Iterator vertices = this.tx().queryVertices(server); + Iterator vertices = this.tx().queryVertices(serverId); Vertex vertex = QueryResults.one(vertices); if (vertex == null) { return null; diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/StandardTaskScheduler.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/StandardTaskScheduler.java index 9eda3f6b02..120aeb0d66 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/StandardTaskScheduler.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/StandardTaskScheduler.java @@ -141,7 +141,7 @@ private TaskTransaction tx() { @Override public void restoreTasks() { - Id selfServer = this.serverManager().selfServerId(); + Id selfServer = this.serverManager().selfNodeId(); // Restore 'RESTORING', 'RUNNING' and 'QUEUED' tasks in order. for (TaskStatus status : TaskStatus.PENDING_STATUSES) { String page = this.supportsPaging() ? PageInfo.PAGE_NONE : null; @@ -177,35 +177,35 @@ private Future restore(HugeTask task) { public Future schedule(HugeTask task) { E.checkArgumentNotNull(task, "Task can't be null"); + /* + * Just submit to queue if status=QUEUED (means re-schedule task) + * NOTE: schedule() method may be called multi times by + * HugeTask.checkDependenciesSuccess() method + */ if (task.status() == TaskStatus.QUEUED) { - /* - * Just submit to queue if status=QUEUED (means re-schedule task) - * NOTE: schedule() method may be called multi times by - * HugeTask.checkDependenciesSuccess() method - */ return this.resubmitTask(task); } + /* + * Due to EphemeralJob won't be serialized and deserialized through + * shared storage, submit EphemeralJob immediately on any node + */ if (task.callable() instanceof EphemeralJob) { - /* - * Due to EphemeralJob won't be serialized and deserialized through - * shared storage, submit EphemeralJob immediately on master - * NOTE: don't need to save EphemeralJob task - */ + // NOTE: we don't need to save EphemeralJob task task.status(TaskStatus.QUEUED); return this.submitTask(task); } - // Only check if not EphemeralJob + // Check this is on master for normal task schedule this.checkOnMasterNode("schedule"); if (this.serverManager().onlySingleNode() && !task.computer()) { /* * Speed up for single node, submit task immediately, - * this code can be removed without affecting logic + * this code can be removed without affecting code logic */ task.status(TaskStatus.QUEUED); - task.server(this.serverManager().selfServerId()); + task.server(this.serverManager().selfNodeId()); this.save(task); return this.submitTask(task); } else { @@ -278,8 +278,8 @@ public synchronized void cancel(HugeTask task) { // The task scheduled to workers, let the worker node to cancel this.save(task); assert task.server() != null : task; - assert this.serverManager().master(); - if (!task.server().equals(this.serverManager().selfServerId())) { + assert this.serverManager().selfIsMaster(); + if (!task.server().equals(this.serverManager().selfNodeId())) { /* * Remove task from memory if it's running on worker node, * but keep task in memory if it's running on master node. @@ -303,10 +303,10 @@ protected ServerInfoManager serverManager() { return this.serverManager; } - protected synchronized void scheduleTasks() { + protected synchronized void scheduleTasksOnMaster() { // Master server schedule all scheduling tasks to suitable worker nodes - Collection scheduleInfos = this.serverManager() - .allServerInfos(); + Collection serverInfos = this.serverManager() + .allServerInfos(); String page = this.supportsPaging() ? PageInfo.PAGE_NONE : null; do { Iterator> tasks = this.tasks(TaskStatus.SCHEDULING, @@ -318,12 +318,12 @@ protected synchronized void scheduleTasks() { continue; } - if (!this.serverManager.master()) { + if (!this.serverManager.selfIsMaster()) { return; } HugeServerInfo server = this.serverManager().pickWorkerNode( - scheduleInfos, task); + serverInfos, task); if (server == null) { LOG.info("The master can't find suitable servers to " + "execute task '{}', wait for next schedule", @@ -348,7 +348,8 @@ protected synchronized void scheduleTasks() { } } while (page != null); - this.serverManager().updateServerInfos(scheduleInfos); + // Save to store + this.serverManager().updateServerInfos(serverInfos); } protected void executeTasksOnWorker(Id server) { @@ -422,7 +423,7 @@ protected void cancelTasksOnWorker(Id server) { protected void taskDone(HugeTask task) { this.remove(task); - Id selfServerId = this.serverManager().selfServerId(); + Id selfServerId = this.serverManager().selfNodeId(); try { this.serverManager().decreaseLoad(task.load()); } catch (Throwable e) { @@ -718,7 +719,7 @@ private V call(Callable callable) { } private void checkOnMasterNode(String op) { - if (!this.serverManager().master()) { + if (!this.serverManager().selfIsMaster()) { throw new HugeException("Can't %s task on non-master server", op); } } diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/TaskManager.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/TaskManager.java index 0ad96f443c..b3726d3830 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/TaskManager.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/task/TaskManager.java @@ -243,19 +243,8 @@ public int pendingTasks() { return size; } - protected void notifyNewTask(HugeTask task) { - Queue queue = ((ThreadPoolExecutor) this.schedulerExecutor) - .getQueue(); - if (queue.size() <= 1) { - /* - * Notify to schedule tasks initiatively when have new task - * It's OK to not notify again if there are more than one task in - * queue(like two, one is timer task, one is immediate task), - * we don't want too many immediate tasks to be inserted into queue, - * one notify will cause all the tasks to be processed. - */ - this.schedulerExecutor.submit(this::scheduleOrExecuteJob); - } + public void enableRoleElection() { + this.enableRoleElected = true; } public void onAsRoleMaster() { @@ -263,7 +252,7 @@ public void onAsRoleMaster() { for (TaskScheduler entry : this.schedulers.values()) { StandardTaskScheduler scheduler = (StandardTaskScheduler) entry; ServerInfoManager serverInfoManager = scheduler.serverManager(); - serverInfoManager.forceInitServerInfo(serverInfoManager.selfServerId(), NodeRole.MASTER); + serverInfoManager.changeServerRole(NodeRole.MASTER); } } catch (Throwable e) { LOG.error("Exception occurred when change to master role", e); @@ -276,7 +265,7 @@ public void onAsRoleWorker() { for (TaskScheduler entry : this.schedulers.values()) { StandardTaskScheduler scheduler = (StandardTaskScheduler) entry; ServerInfoManager serverInfoManager = scheduler.serverManager(); - serverInfoManager.forceInitServerInfo(serverInfoManager.selfServerId(), NodeRole.WORKER); + serverInfoManager.changeServerRole(NodeRole.WORKER); } } catch (Throwable e) { LOG.error("Exception occurred when change to worker role", e); @@ -284,8 +273,19 @@ public void onAsRoleWorker() { } } - public void enableRoleElected(boolean enableRoleElected) { - this.enableRoleElected = enableRoleElected; + protected void notifyNewTask(HugeTask task) { + Queue queue = ((ThreadPoolExecutor) this.schedulerExecutor) + .getQueue(); + if (queue.size() <= 1) { + /* + * Notify to schedule tasks initiatively when have new task + * It's OK to not notify again if there are more than one task in + * queue(like two, one is timer task, one is immediate task), + * we don't want too many immediate tasks to be inserted into queue, + * one notify will cause all the tasks to be processed. + */ + this.schedulerExecutor.submit(this::scheduleOrExecuteJob); + } } private void scheduleOrExecuteJob() { @@ -324,7 +324,7 @@ private void scheduleOrExecuteJobForGraph(StandardTaskScheduler scheduler) { * If graph is closed, don't call serverManager.initialized() * due to it will reopen graph tx. */ - if (!serverManager.graphReady()) { + if (!serverManager.graphIsReady()) { return; } @@ -332,25 +332,25 @@ private void scheduleOrExecuteJobForGraph(StandardTaskScheduler scheduler) { serverManager.heartbeat(); /* - * Master schedule tasks to suitable servers. - * Worker maybe become Master, so Master also need perform tasks assigned by - * previous Master when enableRoleElected is true. - * However, the master only needs to take the assignment, - * because the master stays the same when enableRoleElected is false. - * There is no suitable server when these tasks are created + * Master will schedule tasks to suitable servers. + * Note a Worker may become to a Master, so elected-Master also needs to + * execute tasks assigned by previous Master when enableRoleElected=true. + * However, when enableRoleElected=false, a Master is only set by the + * config assignment, assigned-Master always stays the same state. */ - if (serverManager.master()) { - scheduler.scheduleTasks(); + if (serverManager.selfIsMaster()) { + scheduler.scheduleTasksOnMaster(); if (!this.enableRoleElected && !serverManager.onlySingleNode()) { + // assigned-Master + non-single-node don't need to execute tasks return; } } - // Schedule queued tasks scheduled to current server - scheduler.executeTasksOnWorker(serverManager.selfServerId()); + // Execute queued tasks scheduled to current server + scheduler.executeTasksOnWorker(serverManager.selfNodeId()); // Cancel tasks scheduled to current server - scheduler.cancelTasksOnWorker(serverManager.selfServerId()); + scheduler.cancelTasksOnWorker(serverManager.selfNodeId()); } finally { LockUtil.unlock(graph, LockUtil.GRAPH_LOCK); } diff --git a/hugegraph-server/hugegraph-dist/src/assembly/static/conf/gremlin-server.yaml b/hugegraph-server/hugegraph-dist/src/assembly/static/conf/gremlin-server.yaml index dff43cb04b..5946779828 100644 --- a/hugegraph-server/hugegraph-dist/src/assembly/static/conf/gremlin-server.yaml +++ b/hugegraph-server/hugegraph-dist/src/assembly/static/conf/gremlin-server.yaml @@ -40,6 +40,8 @@ scriptEngines: { org.apache.hugegraph.backend.id.IdGenerator, org.apache.hugegraph.type.define.Directions, org.apache.hugegraph.type.define.NodeRole, + org.apache.hugegraph.masterelection.GlobalMasterInfo, + org.apache.hugegraph.util.DateUtil, org.apache.hugegraph.traversal.algorithm.CollectionPathsTraverser, org.apache.hugegraph.traversal.algorithm.CountTraverser, org.apache.hugegraph.traversal.algorithm.CustomizedCrosspointsTraverser, @@ -64,7 +66,6 @@ scriptEngines: { org.apache.hugegraph.traversal.optimize.ConditionP, org.apache.hugegraph.traversal.optimize.Text, org.apache.hugegraph.traversal.optimize.TraversalUtil, - org.apache.hugegraph.util.DateUtil, org.opencypher.gremlin.traversal.CustomFunctions, org.opencypher.gremlin.traversal.CustomPredicate ], diff --git a/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft1/gremlin-server.yaml b/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft1/gremlin-server.yaml index 517c0a5fcc..55a8c0bb34 100644 --- a/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft1/gremlin-server.yaml +++ b/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft1/gremlin-server.yaml @@ -36,6 +36,8 @@ scriptEngines: { org.apache.hugegraph.backend.id.IdGenerator, org.apache.hugegraph.type.define.Directions, org.apache.hugegraph.type.define.NodeRole, + org.apache.hugegraph.masterelection.GlobalMasterInfo, + org.apache.hugegraph.util.DateUtil, org.apache.hugegraph.traversal.algorithm.CollectionPathsTraverser, org.apache.hugegraph.traversal.algorithm.CountTraverser, org.apache.hugegraph.traversal.algorithm.CustomizedCrosspointsTraverser, @@ -57,9 +59,11 @@ scriptEngines: { org.apache.hugegraph.traversal.algorithm.steps.EdgeStep, org.apache.hugegraph.traversal.algorithm.steps.RepeatEdgeStep, org.apache.hugegraph.traversal.algorithm.steps.WeightedEdgeStep, + org.apache.hugegraph.traversal.optimize.ConditionP, org.apache.hugegraph.traversal.optimize.Text, org.apache.hugegraph.traversal.optimize.TraversalUtil, - org.apache.hugegraph.util.DateUtil + org.opencypher.gremlin.traversal.CustomFunctions, + org.opencypher.gremlin.traversal.CustomPredicate ], methodImports: [java.lang.Math#*] }, diff --git a/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft2/gremlin-server.yaml b/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft2/gremlin-server.yaml index bd37369c83..54d9a6ddec 100644 --- a/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft2/gremlin-server.yaml +++ b/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft2/gremlin-server.yaml @@ -36,6 +36,8 @@ scriptEngines: { org.apache.hugegraph.backend.id.IdGenerator, org.apache.hugegraph.type.define.Directions, org.apache.hugegraph.type.define.NodeRole, + org.apache.hugegraph.masterelection.GlobalMasterInfo, + org.apache.hugegraph.util.DateUtil, org.apache.hugegraph.traversal.algorithm.CollectionPathsTraverser, org.apache.hugegraph.traversal.algorithm.CountTraverser, org.apache.hugegraph.traversal.algorithm.CustomizedCrosspointsTraverser, @@ -57,9 +59,11 @@ scriptEngines: { org.apache.hugegraph.traversal.algorithm.steps.EdgeStep, org.apache.hugegraph.traversal.algorithm.steps.RepeatEdgeStep, org.apache.hugegraph.traversal.algorithm.steps.WeightedEdgeStep, + org.apache.hugegraph.traversal.optimize.ConditionP, org.apache.hugegraph.traversal.optimize.Text, org.apache.hugegraph.traversal.optimize.TraversalUtil, - org.apache.hugegraph.util.DateUtil + org.opencypher.gremlin.traversal.CustomFunctions, + org.opencypher.gremlin.traversal.CustomPredicate ], methodImports: [java.lang.Math#*] }, diff --git a/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft3/gremlin-server.yaml b/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft3/gremlin-server.yaml index a034d63520..508abef354 100644 --- a/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft3/gremlin-server.yaml +++ b/hugegraph-server/hugegraph-dist/src/assembly/travis/conf-raft3/gremlin-server.yaml @@ -36,6 +36,8 @@ scriptEngines: { org.apache.hugegraph.backend.id.IdGenerator, org.apache.hugegraph.type.define.Directions, org.apache.hugegraph.type.define.NodeRole, + org.apache.hugegraph.masterelection.GlobalMasterInfo, + org.apache.hugegraph.util.DateUtil, org.apache.hugegraph.traversal.algorithm.CollectionPathsTraverser, org.apache.hugegraph.traversal.algorithm.CountTraverser, org.apache.hugegraph.traversal.algorithm.CustomizedCrosspointsTraverser, @@ -57,9 +59,11 @@ scriptEngines: { org.apache.hugegraph.traversal.algorithm.steps.EdgeStep, org.apache.hugegraph.traversal.algorithm.steps.RepeatEdgeStep, org.apache.hugegraph.traversal.algorithm.steps.WeightedEdgeStep, + org.apache.hugegraph.traversal.optimize.ConditionP, org.apache.hugegraph.traversal.optimize.Text, org.apache.hugegraph.traversal.optimize.TraversalUtil, - org.apache.hugegraph.util.DateUtil + org.opencypher.gremlin.traversal.CustomFunctions, + org.opencypher.gremlin.traversal.CustomPredicate ], methodImports: [java.lang.Math#*] }, diff --git a/hugegraph-server/hugegraph-example/src/main/java/org/apache/hugegraph/example/ExampleUtil.java b/hugegraph-server/hugegraph-example/src/main/java/org/apache/hugegraph/example/ExampleUtil.java index 7cb148262a..8317f9ba7f 100644 --- a/hugegraph-server/hugegraph-example/src/main/java/org/apache/hugegraph/example/ExampleUtil.java +++ b/hugegraph-server/hugegraph-example/src/main/java/org/apache/hugegraph/example/ExampleUtil.java @@ -20,18 +20,17 @@ import java.io.File; import java.util.Iterator; import java.util.concurrent.TimeoutException; -import org.slf4j.Logger; import org.apache.hugegraph.HugeException; import org.apache.hugegraph.HugeFactory; import org.apache.hugegraph.HugeGraph; -import org.apache.hugegraph.backend.id.IdGenerator; import org.apache.hugegraph.dist.RegisterUtil; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.perf.PerfUtil; import org.apache.hugegraph.task.HugeTask; import org.apache.hugegraph.task.TaskScheduler; -import org.apache.hugegraph.type.define.NodeRole; import org.apache.hugegraph.util.Log; +import org.slf4j.Logger; public class ExampleUtil { private static final Logger LOG = Log.logger(ExampleUtil.class); @@ -81,7 +80,7 @@ public static HugeGraph loadGraph(boolean needClear, boolean needProfile) { graph.clearBackend(); } graph.initBackend(); - graph.serverStarted(IdGenerator.of("server1"), NodeRole.MASTER); + graph.serverStarted(GlobalMasterInfo.master("server1")); return graph; } diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/api/GremlinApiTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/api/GremlinApiTest.java index 57aefeb9be..b065270871 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/api/GremlinApiTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/api/GremlinApiTest.java @@ -121,7 +121,7 @@ public void testClearAndInit() { body = "{" + "\"gremlin\":\"hugegraph.serverStarted(" + - " IdGenerator.of('server1'), NodeRole.MASTER)\"," + + " GlobalMasterInfo.master('server1'))\"," + "\"bindings\":{}," + "\"language\":\"gremlin-groovy\"," + "\"aliases\":{\"g\":\"__g_hugegraph\"}}"; diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/BaseCoreTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/BaseCoreTest.java index d51e1b5951..df9932ab8e 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/BaseCoreTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/BaseCoreTest.java @@ -21,13 +21,12 @@ import org.apache.hugegraph.HugeGraph; import org.apache.hugegraph.HugeGraphParams; -import org.apache.hugegraph.backend.id.IdGenerator; import org.apache.hugegraph.backend.store.BackendFeatures; import org.apache.hugegraph.dist.RegisterUtil; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.schema.SchemaManager; import org.apache.hugegraph.testutil.Utils; import org.apache.hugegraph.testutil.Whitebox; -import org.apache.hugegraph.type.define.NodeRole; import org.apache.hugegraph.util.Log; import org.apache.tinkerpop.gremlin.structure.Edge; import org.apache.tinkerpop.gremlin.structure.Vertex; @@ -66,7 +65,7 @@ public static void init() { graph = Utils.open(); graph.clearBackend(); graph.initBackend(); - graph.serverStarted(IdGenerator.of("server1"), NodeRole.MASTER); + graph.serverStarted(GlobalMasterInfo.master("server-test")); } @AfterClass diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/CoreTestSuite.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/CoreTestSuite.java index e140474b40..beafedf863 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/CoreTestSuite.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/CoreTestSuite.java @@ -36,9 +36,7 @@ TaskCoreTest.class, AuthTest.class, MultiGraphsTest.class, - RamTableTest.class, - RoleElectionStateMachineTest.class + RamTableTest.class }) public class CoreTestSuite { - } diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/MultiGraphsTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/MultiGraphsTest.java index 23fb122f44..3b468ba458 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/MultiGraphsTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/MultiGraphsTest.java @@ -31,8 +31,8 @@ import org.apache.hugegraph.backend.store.BackendStoreInfo; import org.apache.hugegraph.backend.store.rocksdb.RocksDBOptions; import org.apache.hugegraph.config.CoreOptions; -import org.apache.hugegraph.exception.ConnectionException; import org.apache.hugegraph.exception.ExistedException; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.schema.EdgeLabel; import org.apache.hugegraph.schema.IndexLabel; import org.apache.hugegraph.schema.PropertyKey; @@ -40,7 +40,6 @@ import org.apache.hugegraph.schema.VertexLabel; import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.testutil.Utils; -import org.apache.hugegraph.type.define.NodeRole; import org.apache.tinkerpop.gremlin.structure.T; import org.apache.tinkerpop.gremlin.structure.Vertex; import org.apache.tinkerpop.gremlin.structure.util.GraphFactory; @@ -87,9 +86,9 @@ public void testCopySchemaWithMultiGraphs() { graph.initBackend(); } HugeGraph g1 = graphs.get(0); - g1.serverStarted(IdGenerator.of("server-g2"), NodeRole.MASTER); + g1.serverStarted(GlobalMasterInfo.master("server-g2")); HugeGraph g2 = graphs.get(1); - g2.serverStarted(IdGenerator.of("server-g3"), NodeRole.MASTER); + g2.serverStarted(GlobalMasterInfo.master("server-g3")); SchemaManager schema = g1.schema(); @@ -209,8 +208,8 @@ public void testCopySchemaWithMultiGraphsWithConflict() { } HugeGraph g1 = graphs.get(0); HugeGraph g2 = graphs.get(1); - g1.serverStarted(IdGenerator.of("server-g1c"), NodeRole.MASTER); - g2.serverStarted(IdGenerator.of("server-g2c"), NodeRole.MASTER); + g1.serverStarted(GlobalMasterInfo.master("server-g1c")); + g2.serverStarted(GlobalMasterInfo.master("server-g2c")); g1.schema().propertyKey("id").asInt().create(); g2.schema().propertyKey("id").asText().create(); diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/RoleElectionStateMachineTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/RoleElectionStateMachineTest.java index 9fbbb5e628..dd73821661 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/RoleElectionStateMachineTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/RoleElectionStateMachineTest.java @@ -34,8 +34,8 @@ import org.apache.hugegraph.masterelection.ClusterRoleStore; import org.apache.hugegraph.masterelection.Config; import org.apache.hugegraph.masterelection.RoleElectionStateMachine; +import org.apache.hugegraph.masterelection.RoleListener; import org.apache.hugegraph.masterelection.StandardRoleElectionStateMachine; -import org.apache.hugegraph.masterelection.StateMachineCallback; import org.apache.hugegraph.masterelection.StateMachineContext; import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.testutil.Utils; @@ -43,13 +43,13 @@ public class RoleElectionStateMachineTest { - public static class LogEntry { + private static class LogEntry { - Integer epoch; + private final Integer epoch; - String node; + private final String node; - Role role; + private final Role role; enum Role { master, @@ -74,28 +74,29 @@ public boolean equals(Object obj) { return false; } LogEntry logEntry = (LogEntry) obj; - return Objects.equals(epoch, logEntry.epoch) && - Objects.equals(node, logEntry.node) && role == logEntry.role; + return Objects.equals(this.epoch, logEntry.epoch) && + Objects.equals(this.node, logEntry.node) && + this.role == logEntry.role; } @Override public int hashCode() { - return Objects.hash(epoch, node, role); + return Objects.hash(this.epoch, this.node, this.role); } @Override public String toString() { return "LogEntry{" + - "epoch=" + epoch + - ", node='" + node + '\'' + - ", role=" + role + + "epoch=" + this.epoch + + ", node='" + this.node + '\'' + + ", role=" + this.role + '}'; } } private static class TestConfig implements Config { - String node; + private final String node; public TestConfig(String node) { this.node = node; @@ -139,11 +140,11 @@ public long baseTimeoutMillisecond() { @Test public void testStateMachine() throws InterruptedException { - final CountDownLatch stop = new CountDownLatch(4); final int MAX_COUNT = 200; - final List logRecords = Collections.synchronizedList(new ArrayList<>(MAX_COUNT)); - final List masterNodes = Collections.synchronizedList(new ArrayList<>(MAX_COUNT)); - final StateMachineCallback callback = new StateMachineCallback() { + CountDownLatch stop = new CountDownLatch(4); + List logRecords = Collections.synchronizedList(new ArrayList<>(MAX_COUNT)); + List masterNodes = Collections.synchronizedList(new ArrayList<>(MAX_COUNT)); + RoleListener callback = new RoleListener() { @Override public void onAsRoleMaster(StateMachineContext context) { @@ -200,12 +201,13 @@ public void onAsRoleAbdication(StateMachineContext context) { @Override public void error(StateMachineContext context, Throwable e) { Utils.println("state machine error: node " + - context.node() + - " message " + e.getMessage()); + context.node() + " message " + e.getMessage()); } }; - final List clusterRoleLogs = Collections.synchronizedList(new ArrayList<>(100)); + final List clusterRoleLogs = Collections.synchronizedList( + new ArrayList<>(100)); + final ClusterRoleStore clusterRoleStore = new ClusterRoleStore() { volatile int epoch = 0; @@ -227,7 +229,7 @@ public boolean updateIfNodePresent(ClusterRole clusterRole) { } ClusterRole copy = this.copy(clusterRole); - ClusterRole newClusterRole = data.compute(copy.epoch(), (key, value) -> { + ClusterRole newClusterRole = this.data.compute(copy.epoch(), (key, value) -> { if (copy.epoch() > this.epoch) { this.epoch = copy.epoch(); Assert.assertNull(value); @@ -262,27 +264,27 @@ public Optional query() { Thread node1 = new Thread(() -> { Config config = new TestConfig("1"); RoleElectionStateMachine stateMachine = - new StandardRoleElectionStateMachine(config, clusterRoleStore); + new StandardRoleElectionStateMachine(config, clusterRoleStore); machines[1] = stateMachine; - stateMachine.apply(callback); + stateMachine.start(callback); stop.countDown(); }); Thread node2 = new Thread(() -> { Config config = new TestConfig("2"); RoleElectionStateMachine stateMachine = - new StandardRoleElectionStateMachine(config, clusterRoleStore); + new StandardRoleElectionStateMachine(config, clusterRoleStore); machines[2] = stateMachine; - stateMachine.apply(callback); + stateMachine.start(callback); stop.countDown(); }); Thread node3 = new Thread(() -> { Config config = new TestConfig("3"); RoleElectionStateMachine stateMachine = - new StandardRoleElectionStateMachine(config, clusterRoleStore); + new StandardRoleElectionStateMachine(config, clusterRoleStore); machines[3] = stateMachine; - stateMachine.apply(callback); + stateMachine.start(callback); stop.countDown(); }); diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java index c6dcff4a87..415e804626 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java @@ -24,27 +24,24 @@ import java.util.Set; import org.apache.commons.configuration2.Configuration; -import org.apache.tinkerpop.gremlin.process.computer.GraphComputer; -import org.apache.tinkerpop.gremlin.structure.Edge; -import org.apache.tinkerpop.gremlin.structure.Graph; -import org.apache.tinkerpop.gremlin.structure.T; -import org.apache.tinkerpop.gremlin.structure.Transaction; -import org.apache.tinkerpop.gremlin.structure.Vertex; -import org.apache.tinkerpop.gremlin.structure.io.Io; - import org.apache.hugegraph.HugeGraph; -import org.apache.hugegraph.backend.id.Id; -import org.apache.hugegraph.backend.id.IdGenerator; import org.apache.hugegraph.backend.store.BackendStoreInfo; import org.apache.hugegraph.io.HugeGraphIoRegistry; import org.apache.hugegraph.io.HugeGraphSONModule; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.perf.PerfUtil.Watched; import org.apache.hugegraph.schema.PropertyKey; import org.apache.hugegraph.schema.SchemaManager; import org.apache.hugegraph.task.TaskScheduler; import org.apache.hugegraph.testutil.Whitebox; import org.apache.hugegraph.type.define.IdStrategy; -import org.apache.hugegraph.type.define.NodeRole; +import org.apache.tinkerpop.gremlin.process.computer.GraphComputer; +import org.apache.tinkerpop.gremlin.structure.Edge; +import org.apache.tinkerpop.gremlin.structure.Graph; +import org.apache.tinkerpop.gremlin.structure.T; +import org.apache.tinkerpop.gremlin.structure.Transaction; +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.apache.tinkerpop.gremlin.structure.io.Io; import com.google.common.collect.ImmutableSet; @@ -85,8 +82,7 @@ protected void initBackend() { assert sysInfo.exists() && !this.graph.closed(); } - Id id = IdGenerator.of("server-tinkerpop"); - this.graph.serverStarted(id, NodeRole.MASTER); + this.graph.serverStarted(GlobalMasterInfo.master("server-tinkerpop")); this.initedBackend = true; } diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/UnitTestSuite.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/UnitTestSuite.java index eb1fb6ad3b..458c2d8a9e 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/UnitTestSuite.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/UnitTestSuite.java @@ -17,6 +17,7 @@ package org.apache.hugegraph.unit; +import org.apache.hugegraph.core.RoleElectionStateMachineTest; import org.apache.hugegraph.unit.cache.CacheManagerTest; import org.apache.hugegraph.unit.cache.CacheTest; import org.apache.hugegraph.unit.cache.CachedGraphTransactionTest; @@ -111,6 +112,7 @@ TraversalUtilTest.class, PageStateTest.class, SystemSchemaStoreTest.class, + RoleElectionStateMachineTest.class, /* serializer */ BytesBufferTest.class, diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/core/SecurityManagerTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/core/SecurityManagerTest.java index 403bc62e99..ae431480c4 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/core/SecurityManagerTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/core/SecurityManagerTest.java @@ -32,23 +32,22 @@ import java.util.Map; import java.util.concurrent.TimeoutException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - import org.apache.hugegraph.HugeException; import org.apache.hugegraph.HugeFactory; import org.apache.hugegraph.HugeGraph; -import org.apache.hugegraph.backend.id.IdGenerator; import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.job.GremlinJob; import org.apache.hugegraph.job.JobBuilder; +import org.apache.hugegraph.masterelection.GlobalMasterInfo; import org.apache.hugegraph.security.HugeSecurityManager; import org.apache.hugegraph.task.HugeTask; import org.apache.hugegraph.testutil.Assert; -import org.apache.hugegraph.type.define.NodeRole; import org.apache.hugegraph.unit.FakeObjects; import org.apache.hugegraph.util.JsonUtil; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + import com.google.common.collect.ImmutableMap; public class SecurityManagerTest { @@ -319,7 +318,7 @@ private static HugeGraph loadGraph(boolean needClear) { graph.clearBackend(); } graph.initBackend(); - graph.serverStarted(IdGenerator.of("server1"), NodeRole.MASTER); + graph.serverStarted(GlobalMasterInfo.master("server1")); return graph; } From e8fb2696d6154f20fd4d6e45ca58763e69145116 Mon Sep 17 00:00:00 2001 From: Simon Cheung Date: Tue, 5 Dec 2023 21:50:02 +0800 Subject: [PATCH 2/9] chore: fix curl failed to request https urls (#2378) --- hugegraph-server/hugegraph-dist/src/assembly/static/bin/util.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hugegraph-server/hugegraph-dist/src/assembly/static/bin/util.sh b/hugegraph-server/hugegraph-dist/src/assembly/static/bin/util.sh index 64980403b1..fa3f94a215 100755 --- a/hugegraph-server/hugegraph-dist/src/assembly/static/bin/util.sh +++ b/hugegraph-server/hugegraph-dist/src/assembly/static/bin/util.sh @@ -141,7 +141,7 @@ function wait_for_startup() { return 1 fi - status=$(curl -I -s -w "%{http_code}" -o /dev/null "$server_url") + status=$(curl -I -s -k -w "%{http_code}" -o /dev/null "$server_url") if [[ $status -eq 200 || $status -eq 401 ]]; then echo "OK" echo "Started [pid $pid]" From 47aa8be8508293bbda76c93b461292efc84a75c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B0=8F=E5=AE=87?= <940643974@qq.com> Date: Thu, 7 Dec 2023 12:26:22 +0800 Subject: [PATCH 3/9] refact(api): update common 1.2 & fix jersey client code problem (#2365) * update common version and fix rest client problem * use stage * fix dependency issue * test * test * test * test * fix * chore: improve the ci logic * fix core-version * Empty test * fix code issue * fix: 3rd party changes * code optimize * refactor the AbsJerseyRestClient * fix code issue --------- Co-authored-by: imbajin --- .github/configs/settings.xml | 2 + .github/workflows/check-dependencies.yml | 6 +- .github/workflows/ci.yml | 43 +++-- .github/workflows/codeql-analysis.yml | 6 +- .github/workflows/licence-checker.yml | 2 + .../api/gremlin/AbstractJerseyRestClient.java | 158 ++++++++++++++++++ .../hugegraph/api/gremlin/GremlinClient.java | 66 +++++--- .../apache/hugegraph/version/ApiVersion.java | 5 + hugegraph-server/hugegraph-core/pom.xml | 28 +++- .../apache/hugegraph/version/CoreVersion.java | 24 +-- .../scripts/dependency/check_dependencies.sh | 17 +- .../scripts/dependency/known-dependencies.txt | 12 +- .../src/assembly/travis/install-backend.sh | 2 + .../src/assembly/travis/install-cassandra.sh | 1 - .../src/assembly/travis/install-hbase.sh | 2 +- .../backend/store/palo/PaloHttpClient.java | 15 +- hugegraph-server/pom.xml | 2 +- 17 files changed, 301 insertions(+), 90 deletions(-) create mode 100644 hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/AbstractJerseyRestClient.java diff --git a/.github/configs/settings.xml b/.github/configs/settings.xml index 3fcc52dea3..294ded1cb2 100644 --- a/.github/configs/settings.xml +++ b/.github/configs/settings.xml @@ -5,7 +5,9 @@ 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. diff --git a/.github/workflows/check-dependencies.yml b/.github/workflows/check-dependencies.yml index 119d55bb52..5350d53fe6 100644 --- a/.github/workflows/check-dependencies.yml +++ b/.github/workflows/check-dependencies.yml @@ -5,7 +5,7 @@ on: branches: - /^release-.*$/ pull_request: - + permissions: contents: read @@ -17,7 +17,7 @@ jobs: SCRIPT_DEPENDENCY: hugegraph-server/hugegraph-dist/scripts/dependency steps: - name: Checkout source - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Set up JDK 11 uses: actions/setup-java@v3 with: @@ -44,7 +44,7 @@ jobs: runs-on: ubuntu-latest steps: - name: 'Checkout Repository' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: 'Dependency Review' uses: actions/dependency-review-action@v3 # Refer: https://github.com/actions/dependency-review-action diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index f6d276e90a..5907bffa30 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -30,48 +30,44 @@ jobs: JAVA_VERSION: [ '8', '11' ] steps: - - name: Install JDK ${{ matrix.JAVA_VERSION }} - uses: actions/setup-java@v3 - with: - java-version: ${{ matrix.JAVA_VERSION }} - distribution: 'zulu' - - - name: Cache Maven packages - uses: actions/cache@v3 - with: - path: ~/.m2 - key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} - restore-keys: ${{ runner.os }}-m2 - - name: Checkout uses: actions/checkout@v4 with: fetch-depth: 2 - - name: Compile - run: | - mvn clean compile -U -Dmaven.javadoc.skip=true -ntp - - - name: Install JDK 8 + # TODO: Remove this step after install-backend.sh updated + - name: Install Java8 for backend uses: actions/setup-java@v3 with: java-version: '8' distribution: 'zulu' - - name: Prepare env and service + + - name: Prepare backend environment run: | - $TRAVIS_DIR/install-backend.sh $BACKEND + $TRAVIS_DIR/install-backend.sh $BACKEND && jps -l - - name: Install JDK ${{ matrix.JAVA_VERSION }} + - name: Install Java ${{ matrix.JAVA_VERSION }} uses: actions/setup-java@v3 with: java-version: ${{ matrix.JAVA_VERSION }} distribution: 'zulu' - - name: use staged maven repo settings + - name: Cache Maven packages + uses: actions/cache@v3 + with: + path: ~/.m2 + key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} + restore-keys: ${{ runner.os }}-m2 + + - name: Use staged maven repo settings if: ${{ env.USE_STAGE == 'true' }} run: | cp $HOME/.m2/settings.xml /tmp/settings.xml - mv -vf .github/configs/settings.xml $HOME/.m2/settings.xml + cp -vf .github/configs/settings.xml $HOME/.m2/settings.xml && cat $HOME/.m2/settings.xml + + - name: Compile + run: | + mvn clean compile -U -Dmaven.javadoc.skip=true -ntp - name: Run unit test run: | @@ -99,4 +95,5 @@ jobs: - name: Upload coverage to Codecov uses: codecov/codecov-action@v3 with: + token: ${{ secrets.CODECOV_TOKEN }} file: ${{ env.REPORT_DIR }}/*.xml diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 958e5b1bdc..9165bfda94 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -4,8 +4,8 @@ on: push: branches: [ master, release-* ] pull_request: - # The branches below must be a subset of the branches above - # branches: [ master ] # enable in all PR + # The branches below must be a subset of the branches above + # branches: [ master ] # enable in all PRs schedule: - cron: '33 0 * * 5' @@ -27,7 +27,7 @@ jobs: steps: - name: Checkout repository - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Setup Java JDK uses: actions/setup-java@v3 diff --git a/.github/workflows/licence-checker.yml b/.github/workflows/licence-checker.yml index 3d14cc0620..2510b44de1 100644 --- a/.github/workflows/licence-checker.yml +++ b/.github/workflows/licence-checker.yml @@ -10,6 +10,8 @@ on: jobs: check-license: runs-on: ubuntu-latest + env: + USE_STAGE: 'true' # Whether to include the stage repository. steps: - uses: actions/checkout@v4 diff --git a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/AbstractJerseyRestClient.java b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/AbstractJerseyRestClient.java new file mode 100644 index 0000000000..181abc46d5 --- /dev/null +++ b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/AbstractJerseyRestClient.java @@ -0,0 +1,158 @@ +/* + * 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 org.apache.hugegraph.api.gremlin; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.apache.http.pool.PoolStats; +import org.apache.hugegraph.util.ExecutorUtil; +import org.glassfish.jersey.apache.connector.ApacheClientProperties; +import org.glassfish.jersey.apache.connector.ApacheConnectorProvider; +import org.glassfish.jersey.client.ClientConfig; +import org.glassfish.jersey.client.ClientProperties; +import org.glassfish.jersey.client.JerseyClientBuilder; +import org.glassfish.jersey.message.GZipEncoder; + +import jakarta.ws.rs.client.Client; +import jakarta.ws.rs.client.WebTarget; + +/** + * This class is a simplified class of AbstractRestClient from hugegraph-common. + * For some reason, we replace the rest implementation from jersey to okhttp. + * But GremlinClient still uses jersey-client to forward request, so we copy the old + * AbstractRestClient from hugegraph-common and rename the name to AbstractJerseyRestClient. + * Because we don't need the full feature of AbstractRestClient, so we reduce some useless code. + */ +public abstract class AbstractJerseyRestClient { + + /** + * Time unit: hour + */ + private static final long TTL = 24L; + /** + * Time unit: ms + */ + private static final long IDLE_TIME = 40L * 1000L; + private static final String PROPERTY_MAX_TOTAL = "maxTotal"; + private static final String PROPERTY_MAX_PER_ROUTE = "maxPerRoute"; + private static final String PROPERTY_IDLE_TIME = "idleTime"; + private static final String CONNECTION_MANAGER = ApacheClientProperties.CONNECTION_MANAGER; + + private final Client client; + private final WebTarget webTarget; + private final PoolingHttpClientConnectionManager pool; + private ScheduledExecutorService cleanExecutor; + + public AbstractJerseyRestClient(String url, int timeout, int maxTotal, int maxPerRoute) { + this(url, new ConfigBuilder().configTimeout(timeout) + .configPool(maxTotal, maxPerRoute) + .build()); + } + + public AbstractJerseyRestClient(String url, ClientConfig config) { + this.pool = configConnectionManager(config); + + this.client = JerseyClientBuilder.newClient(config); + this.client.register(GZipEncoder.class); + this.webTarget = this.client.target(url); + + cleanThreadPoolExecutor(config); + } + + private static PoolingHttpClientConnectionManager configConnectionManager(ClientConfig conf) { + /* + * Using httpclient with connection pooling, and configuring the + * jersey connector. But the jersey that has been released in the maven central + * repository seems to have a bug: https://github.com/jersey/jersey/pull/3752 + */ + PoolingHttpClientConnectionManager pool = + new PoolingHttpClientConnectionManager(TTL, TimeUnit.HOURS); + Integer maxTotal = (Integer) conf.getProperty(PROPERTY_MAX_TOTAL); + Integer maxPerRoute = (Integer) conf.getProperty(PROPERTY_MAX_PER_ROUTE); + + if (maxTotal != null) { + pool.setMaxTotal(maxTotal); + } + if (maxPerRoute != null) { + pool.setDefaultMaxPerRoute(maxPerRoute); + } + conf.property(CONNECTION_MANAGER, pool); + conf.connectorProvider(new ApacheConnectorProvider()); + return pool; + } + + private void cleanThreadPoolExecutor(ClientConfig config) { + this.cleanExecutor = ExecutorUtil.newScheduledThreadPool("conn-clean-worker-%d"); + Number idleTimeProp = (Number) config.getProperty(PROPERTY_IDLE_TIME); + final long idleTime = idleTimeProp == null ? IDLE_TIME : idleTimeProp.longValue(); + final long checkPeriod = idleTime / 2L; + this.cleanExecutor.scheduleWithFixedDelay(() -> { + PoolStats stats = this.pool.getTotalStats(); + int using = stats.getLeased() + stats.getPending(); + if (using > 0) { + // Do clean only when all connections are idle + return; + } + // Release connections when all clients are inactive + this.pool.closeIdleConnections(idleTime, TimeUnit.MILLISECONDS); + this.pool.closeExpiredConnections(); + }, checkPeriod, checkPeriod, TimeUnit.MILLISECONDS); + } + + protected WebTarget getWebTarget() { + return this.webTarget; + } + + public void close() { + try { + if (this.pool != null) { + this.pool.close(); + this.cleanExecutor.shutdownNow(); + } + } finally { + this.client.close(); + } + } + + private static class ConfigBuilder { + + private final ClientConfig config; + + ConfigBuilder() { + this.config = new ClientConfig(); + } + + public ConfigBuilder configTimeout(int timeout) { + this.config.property(ClientProperties.CONNECT_TIMEOUT, timeout); + this.config.property(ClientProperties.READ_TIMEOUT, timeout); + return this; + } + + public ConfigBuilder configPool(int maxTotal, int maxPerRoute) { + this.config.property(PROPERTY_MAX_TOTAL, maxTotal); + this.config.property(PROPERTY_MAX_PER_ROUTE, maxPerRoute); + return this; + } + + public ClientConfig build() { + return this.config; + } + } +} diff --git a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/GremlinClient.java b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/GremlinClient.java index 793884e4da..72af6e8cb1 100644 --- a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/GremlinClient.java +++ b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/gremlin/GremlinClient.java @@ -20,6 +20,9 @@ import java.util.List; import java.util.Map; +import org.apache.hugegraph.api.filter.CompressInterceptor; +import org.apache.hugegraph.util.E; + import jakarta.ws.rs.client.Entity; import jakarta.ws.rs.client.WebTarget; import jakarta.ws.rs.core.HttpHeaders; @@ -27,43 +30,52 @@ import jakarta.ws.rs.core.MultivaluedMap; import jakarta.ws.rs.core.Response; -import org.apache.hugegraph.api.filter.CompressInterceptor; -import org.apache.hugegraph.rest.AbstractRestClient; -import org.apache.hugegraph.testutil.Whitebox; -import org.apache.hugegraph.util.E; - -public class GremlinClient extends AbstractRestClient { - - private final WebTarget webTarget; +/** + * GremlinClient is a client for interacting with a Gremlin server. + * It extends the AbstractJerseyRestClient and provides methods for sending GET and POST requests. + */ +public class GremlinClient extends AbstractJerseyRestClient { - public GremlinClient(String url, int timeout, - int maxTotal, int maxPerRoute) { + /** + * Constructs a GremlinClient with the specified URL, timeout, maxTotal, and maxPerRoute. + * + * @param url The URL of the Gremlin server this client will interact with. + * @param timeout The timeout for the client. + * @param maxTotal The maximum total connections for the client. + * @param maxPerRoute The maximum connections per route for the client. + */ + public GremlinClient(String url, int timeout, int maxTotal, int maxPerRoute) { super(url, timeout, maxTotal, maxPerRoute); - this.webTarget = Whitebox.getInternalState(this, "target"); - E.checkNotNull(this.webTarget, "target"); - } - - @Override - protected void checkStatus(Response response, Response.Status... statuses) { - // pass } + /** + * Sends a POST request to the Gremlin server. + * + * @param auth The authorization token for the request. + * @param req The body of the request. + * @return The response from the server. + */ public Response doPostRequest(String auth, String req) { Entity body = Entity.entity(req, MediaType.APPLICATION_JSON); - return this.webTarget.request() - .header(HttpHeaders.AUTHORIZATION, auth) - .accept(MediaType.APPLICATION_JSON) - .acceptEncoding(CompressInterceptor.GZIP) - .post(body); + return this.getWebTarget().request() + .header(HttpHeaders.AUTHORIZATION, auth) + .accept(MediaType.APPLICATION_JSON) + .acceptEncoding(CompressInterceptor.GZIP) + .post(body); } - public Response doGetRequest(String auth, - MultivaluedMap params) { - WebTarget target = this.webTarget; + /** + * Sends a GET request to the Gremlin server. + * + * @param auth The authorization token for the request. + * @param params The query parameters for the request. + * @return The response from the server. + */ + public Response doGetRequest(String auth, MultivaluedMap params) { + WebTarget target = this.getWebTarget(); for (Map.Entry> entry : params.entrySet()) { E.checkArgument(entry.getValue().size() == 1, - "Invalid query param '%s', can only accept " + - "one value, but got %s", + "Invalid query param '%s', can only accept one value, but got %s", entry.getKey(), entry.getValue()); target = target.queryParam(entry.getKey(), entry.getValue().get(0)); } diff --git a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/version/ApiVersion.java b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/version/ApiVersion.java index a6b7f7c241..c75f65ab82 100644 --- a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/version/ApiVersion.java +++ b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/version/ApiVersion.java @@ -20,6 +20,11 @@ import org.apache.hugegraph.util.VersionUtil; import org.apache.hugegraph.util.VersionUtil.Version; +/** + * This class represents the API version of the HugeGraph system. + * It contains a version number and a method to check the compatibility + * with the core version of the system. + */ public final class ApiVersion { /* diff --git a/hugegraph-server/hugegraph-core/pom.xml b/hugegraph-server/hugegraph-core/pom.xml index de312c9378..bdbb2ace43 100644 --- a/hugegraph-server/hugegraph-core/pom.xml +++ b/hugegraph-server/hugegraph-core/pom.xml @@ -14,8 +14,8 @@ License for the specific language governing permissions and limitations under the License. --> - 4.0.0 @@ -52,6 +52,23 @@ hugegraph-common + + + org.glassfish.jersey.core + jersey-client + ${jersey.version} + + + org.glassfish.jersey.connectors + jersey-apache-connector + ${jersey.version} + + + org.glassfish.jersey.inject + jersey-hk2 + ${jersey.version} + + org.apache.tinkerpop @@ -287,10 +304,13 @@ protobuf-maven-plugin 0.6.1 - com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} + + com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} + protoc-java ${project.basedir}/src/main/resources/proto - ${basedir}/target/generated-sources/protobuf/java + ${basedir}/target/generated-sources/protobuf/java + diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java index 480236ffb6..46b84ebfe9 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java @@ -22,22 +22,24 @@ public class CoreVersion { - static { - // Check versions of the dependency packages - CoreVersion.check(); - } - public static final String NAME = "hugegraph-core"; - public static final String DEFAULT_VERSION = "1.5.0"; - - // The second parameter of Version.of() is for IDE running without JAR + /** + * The second parameter of Version.of() is for IDE running without JAR + */ public static final Version VERSION = Version.of(CoreVersion.class, DEFAULT_VERSION); + /** + * Update it when the gremlin version changed, search "tinkerpop.version" in pom + */ + public static final String GREMLIN_VERSION = "3.5.1"; - public static final String GREMLIN_VERSION = "3.4.3"; + static { + // Check versions of the dependency packages + CoreVersion.check(); + } public static void check() { - // Check version of hugegraph-common - VersionUtil.check(CommonVersion.VERSION, "1.0", "1.1", CommonVersion.NAME); + // Check the version of hugegraph-common + VersionUtil.check(CommonVersion.VERSION, "1.0", "1.35", CommonVersion.NAME); } } diff --git a/hugegraph-server/hugegraph-dist/scripts/dependency/check_dependencies.sh b/hugegraph-server/hugegraph-dist/scripts/dependency/check_dependencies.sh index 309063faa5..024c05e35e 100644 --- a/hugegraph-server/hugegraph-dist/scripts/dependency/check_dependencies.sh +++ b/hugegraph-server/hugegraph-dist/scripts/dependency/check_dependencies.sh @@ -16,17 +16,20 @@ # under the License. # -BASE_PATH=$(cd $(dirname $0); pwd) +BASE_PATH=$( + cd "$(dirname "$0")" || exit 1 + pwd +) # check whether there are new third-party dependencies by diff command, # diff generated 'current-dependencies.txt' file with 'known-dependencies.txt' file. -diff -w -B -U0 <(sort < ${BASE_PATH}/known-dependencies.txt) \ -<(sort < ${BASE_PATH}/current-dependencies.txt) > ${BASE_PATH}/result.txt +diff -w -B -U0 <(sort <"${BASE_PATH}"/known-dependencies.txt) \ + <(sort <"${BASE_PATH}"/current-dependencies.txt) >${BASE_PATH}/result.txt # if has new third-party,the Action will fail and print diff -if [ -s ${BASE_PATH}/result.txt ]; then - cat ${BASE_PATH}/result.txt - exit 1 +if [ -s "${BASE_PATH}"/result.txt ]; then + cat "${BASE_PATH}"/result.txt + exit 1 else - echo 'All third dependencies is known!' + echo 'All third dependencies is known!' fi diff --git a/hugegraph-server/hugegraph-dist/scripts/dependency/known-dependencies.txt b/hugegraph-server/hugegraph-dist/scripts/dependency/known-dependencies.txt index 0069eea761..92e406a122 100644 --- a/hugegraph-server/hugegraph-dist/scripts/dependency/known-dependencies.txt +++ b/hugegraph-server/hugegraph-dist/scripts/dependency/known-dependencies.txt @@ -2,6 +2,7 @@ accessors-smart-1.2.jar airline-0.8.jar animal-sniffer-annotations-1.19.jar annotations-4.1.1.4.jar +annotations-13.0.jar ansj_seg-5.1.6.jar antlr-runtime-3.5.2.jar aopalliance-repackaged-3.0.1.jar @@ -45,6 +46,7 @@ commons-configuration2-2.8.0.jar commons-io-2.7.jar commons-lang-2.6.jar commons-lang3-3.11.jar +commons-lang3-3.12.0.jar commons-logging-1.1.1.jar commons-logging-1.2.jar commons-math3-3.2.jar @@ -127,6 +129,7 @@ jackson-jaxrs-base-2.14.0-rc1.jar jackson-jaxrs-json-provider-2.14.0-rc1.jar jackson-module-jakarta-xmlbind-annotations-2.15.2.jar jackson-module-jaxb-annotations-2.14.0-rc1.jar +jakarta.activation-2.0.0.jar jakarta.activation-2.0.1.jar jakarta.activation-api-1.2.2.jar jakarta.annotation-api-2.0.0.jar @@ -134,7 +137,7 @@ jakarta.inject-api-2.0.0.jar jakarta.servlet-api-5.0.0.jar jakarta.validation-api-3.0.0.jar jakarta.ws.rs-api-3.0.0.jar -jakarta.xml.bind-api-4.0.0-RC2.jar +jakarta.xml.bind-api-3.0.0.jar jamm-0.3.2.jar java-cup-runtime-11b-20160615.jar javapoet-1.8.0.jar @@ -201,10 +204,15 @@ kerby-config-2.0.0.jar kerby-pkix-2.0.0.jar kerby-util-2.0.0.jar kerby-xdr-2.0.0.jar +kotlin-stdlib-1.6.20.jar +kotlin-stdlib-common-1.5.31.jar +kotlin-stdlib-jdk7-1.6.10.jar +kotlin-stdlib-jdk8-1.6.10.jar listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar log4j-api-2.17.1.jar log4j-core-2.17.1.jar log4j-slf4j-impl-2.17.1.jar +logging-interceptor-4.10.0.jar lookout-api-1.4.1.jar lucene-analyzers-common-8.11.2.jar lucene-analyzers-smartcn-8.11.2.jar @@ -232,6 +240,8 @@ nlp-lang-1.7.7.jar objenesis-2.6.jar ohc-core-0.7.4.jar ohc-core-j8-0.5.1.jar +okhttp-4.10.0.jar +okio-jvm-3.0.0.jar opentracing-api-0.22.0.jar opentracing-mock-0.22.0.jar opentracing-noop-0.22.0.jar diff --git a/hugegraph-server/hugegraph-dist/src/assembly/travis/install-backend.sh b/hugegraph-server/hugegraph-dist/src/assembly/travis/install-backend.sh index 64c53a875b..21e2f11e6b 100755 --- a/hugegraph-server/hugegraph-dist/src/assembly/travis/install-backend.sh +++ b/hugegraph-server/hugegraph-dist/src/assembly/travis/install-backend.sh @@ -31,12 +31,14 @@ fi case $BACKEND in cassandra) + # TODO: replace it with docker "$TRAVIS_DIR"/install-cassandra.sh ;; scylladb) "$TRAVIS_DIR"/install-scylladb.sh ;; hbase) + # TODO: replace it with hbase2.3+ to avoid java8 env "$TRAVIS_DIR"/install-hbase.sh ;; mysql) diff --git a/hugegraph-server/hugegraph-dist/src/assembly/travis/install-cassandra.sh b/hugegraph-server/hugegraph-dist/src/assembly/travis/install-cassandra.sh index 2bdfe0bf6a..d83d890dee 100755 --- a/hugegraph-server/hugegraph-dist/src/assembly/travis/install-cassandra.sh +++ b/hugegraph-server/hugegraph-dist/src/assembly/travis/install-cassandra.sh @@ -17,7 +17,6 @@ # set -ev -TRAVIS_DIR=`dirname $0` CASS_DOWNLOAD_ADDRESS="http://archive.apache.org/dist/cassandra" CASS_VERSION="4.0.10" CASS_PACKAGE="apache-cassandra-${CASS_VERSION}" diff --git a/hugegraph-server/hugegraph-dist/src/assembly/travis/install-hbase.sh b/hugegraph-server/hugegraph-dist/src/assembly/travis/install-hbase.sh index 9de48f277e..bdee3e75ab 100755 --- a/hugegraph-server/hugegraph-dist/src/assembly/travis/install-hbase.sh +++ b/hugegraph-server/hugegraph-dist/src/assembly/travis/install-hbase.sh @@ -17,7 +17,7 @@ # set -ev -TRAVIS_DIR=`dirname $0` +TRAVIS_DIR=$(dirname $0) HBASE_DOWNLOAD_ADDRESS="http://archive.apache.org/dist/hbase" HBASE_VERSION="2.0.2" HBASE_PACKAGE="hbase-${HBASE_VERSION}" diff --git a/hugegraph-server/hugegraph-palo/src/main/java/org/apache/hugegraph/backend/store/palo/PaloHttpClient.java b/hugegraph-server/hugegraph-palo/src/main/java/org/apache/hugegraph/backend/store/palo/PaloHttpClient.java index 8ee2f94129..281746e2ee 100644 --- a/hugegraph-server/hugegraph-palo/src/main/java/org/apache/hugegraph/backend/store/palo/PaloHttpClient.java +++ b/hugegraph-server/hugegraph-palo/src/main/java/org/apache/hugegraph/backend/store/palo/PaloHttpClient.java @@ -19,15 +19,15 @@ import java.util.Map; -import jakarta.ws.rs.core.MultivaluedHashMap; -import jakarta.ws.rs.core.MultivaluedMap; -import jakarta.ws.rs.core.Response; - import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.rest.AbstractRestClient; import org.apache.hugegraph.rest.RestClient; +import org.apache.hugegraph.rest.RestHeaders; + import com.google.common.collect.ImmutableMap; +import okhttp3.Response; + public class PaloHttpClient { private final RestClient client; @@ -51,8 +51,8 @@ public void bulkLoadAsync(String table, String body, String label) { // Format path String path = table + "/_load"; // Format headers - MultivaluedMap headers = new MultivaluedHashMap<>(); - headers.putSingle("Expect", "100-continue"); + RestHeaders headers = new RestHeaders(); + headers.add("Expect", "100-continue"); // Format params Map params = ImmutableMap.of("label", label); // Send request @@ -68,8 +68,7 @@ public Client(String url, String user, String password, int timeout) { } @Override - protected void checkStatus(Response response, - Response.Status... statuses) { + protected void checkStatus(Response response, int... statuses) { // pass } } diff --git a/hugegraph-server/pom.xml b/hugegraph-server/pom.xml index 6ab44d9bd0..2d5a51cb99 100644 --- a/hugegraph-server/pom.xml +++ b/hugegraph-server/pom.xml @@ -57,7 +57,7 @@ bash 3.1.2 8.45 - 1.0.0 + 1.2.0 1.47.0 3.21.7 1.36 From c997f35b0475e94a98748dda636cb9bfa78fda77 Mon Sep 17 00:00:00 2001 From: M <87920097+msgui@users.noreply.github.com> Date: Fri, 8 Dec 2023 15:59:28 +0800 Subject: [PATCH 4/9] fix(api): correct the vertex id in the edge-existence api (#2380) --- .../apache/hugegraph/api/traversers/EdgeExistenceAPI.java | 5 +++-- .../traversal/algorithm/EdgeExistenceTraverser.java | 7 +++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/traversers/EdgeExistenceAPI.java b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/traversers/EdgeExistenceAPI.java index 6ffec166e1..4af3ff52fa 100644 --- a/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/traversers/EdgeExistenceAPI.java +++ b/hugegraph-server/hugegraph-api/src/main/java/org/apache/hugegraph/api/traversers/EdgeExistenceAPI.java @@ -22,6 +22,7 @@ import java.util.Iterator; import org.apache.hugegraph.HugeGraph; +import org.apache.hugegraph.api.graph.VertexAPI; import org.apache.hugegraph.backend.id.Id; import org.apache.hugegraph.core.GraphManager; import org.apache.hugegraph.structure.HugeVertex; @@ -72,8 +73,8 @@ public String get(@Context GraphManager manager, E.checkArgumentNotNull(source, "The source can't be null"); E.checkArgumentNotNull(target, "The target can't be null"); - Id sourceId = HugeVertex.getIdValue(source); - Id targetId = HugeVertex.getIdValue(target); + Id sourceId = VertexAPI.checkAndParseVertexId(source); + Id targetId = VertexAPI.checkAndParseVertexId(target); HugeGraph hugegraph = graph(manager, graph); EdgeExistenceTraverser traverser = new EdgeExistenceTraverser(hugegraph); Iterator edges = traverser.queryEdgeExistence(sourceId, targetId, edgeLabel, diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/traversal/algorithm/EdgeExistenceTraverser.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/traversal/algorithm/EdgeExistenceTraverser.java index a7005ad867..38f92daa30 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/traversal/algorithm/EdgeExistenceTraverser.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/traversal/algorithm/EdgeExistenceTraverser.java @@ -42,16 +42,15 @@ public Iterator queryEdgeExistence(Id sourceId, Id targetId, String label, return queryByNeighbors(sourceId, targetId, limit); } - Id edgeLabelId = getEdgeLabelId(label); - EdgeLabel edgeLabel = graph().edgeLabel(edgeLabelId); + EdgeLabel edgeLabel = graph().edgeLabel(label); ConditionQuery conditionQuery = new ConditionQuery(HugeType.EDGE); conditionQuery.eq(HugeKeys.OWNER_VERTEX, sourceId); conditionQuery.eq(HugeKeys.OTHER_VERTEX, targetId); - conditionQuery.eq(HugeKeys.LABEL, edgeLabelId); + conditionQuery.eq(HugeKeys.LABEL, edgeLabel.id()); conditionQuery.eq(HugeKeys.DIRECTION, Directions.OUT); conditionQuery.limit(limit); - if (edgeLabel.existSortKeys()) { + if (edgeLabel.existSortKeys() && !sortValues.isEmpty()) { conditionQuery.eq(HugeKeys.SORT_VALUES, sortValues); } else { conditionQuery.eq(HugeKeys.SORT_VALUES, ""); From bfe9fae150446857412db23ada0dae9d05035837 Mon Sep 17 00:00:00 2001 From: V_Galaxy Date: Sat, 9 Dec 2023 22:02:49 +0800 Subject: [PATCH 5/9] chore: reset hugegraph version to 1.2.0 (#2382) * chore: reset version to 1.2.0 * chore: add README for three submodules * fix: README.md * fix: README.md * fix: README.md * fix: README.md --- hugegraph-pd/README.md | 5 +++++ hugegraph-server/Dockerfile | 2 +- hugegraph-server/README.md | 11 +++++++++++ .../apache/hugegraph/version/CoreVersion.java | 2 +- hugegraph-server/hugegraph-dist/pom.xml | 16 ++++++++++++++++ hugegraph-server/pom.xml | 11 +++++++++++ hugegraph-store/README.md | 5 +++++ pom.xml | 2 +- 8 files changed, 51 insertions(+), 3 deletions(-) create mode 100644 hugegraph-server/README.md diff --git a/hugegraph-pd/README.md b/hugegraph-pd/README.md index e69de29bb2..49548c216d 100644 --- a/hugegraph-pd/README.md +++ b/hugegraph-pd/README.md @@ -0,0 +1,5 @@ +# HugeGraph PD + +HugeGraph PD is a meta server responsible for service discovery, partition information storage, and node scheduling. + +> Note: Currently, the contents of this folder are empty. Starting from revision 1.5.0, the code of HugeGraph PD will be adapted to this location (WIP). diff --git a/hugegraph-server/Dockerfile b/hugegraph-server/Dockerfile index 93368487a9..a28e63ea13 100644 --- a/hugegraph-server/Dockerfile +++ b/hugegraph-server/Dockerfile @@ -26,7 +26,7 @@ RUN mvn package -e -B -ntp -DskipTests -Dmaven.javadoc.skip=true && pwd && ls -l # 2nd stage: runtime env FROM openjdk:11-slim # TODO: get the version from the pom.xml -ENV version=1.5.0 +ENV version=1.2.0 COPY --from=build /pkg/hugegraph-server/apache-hugegraph-incubating-$version/ /hugegraph-server LABEL maintainer="HugeGraph Docker Maintainers " diff --git a/hugegraph-server/README.md b/hugegraph-server/README.md new file mode 100644 index 0000000000..f2c3ceee7f --- /dev/null +++ b/hugegraph-server/README.md @@ -0,0 +1,11 @@ +# HugeGraph Server + +HugeGraph Server consists of two layers of functionality: the graph engine layer, and the storage layer. + +- Graph Engine Layer: + - REST Server: Provides a RESTful API for querying graph/schema information, supports the [Gremlin](https://tinkerpop.apache.org/gremlin.html) and [Cypher](https://en.wikipedia.org/wiki/Cypher) query languages, and offers APIs for service monitoring and operations. + - Graph Engine: Supports both OLTP and OLAP graph computation types, with OLTP implementing the [Apache TinkerPop3](https://tinkerpop.apache.org) framework. + - Backend Interface: Implements the storage of graph data to the backend. + +- Storage Layer: + - Storage Backend: Supports multiple built-in storage backends (RocksDB/MySQL/HBase/...) and allows users to extend custom backends without modifying the existing source code. diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java index 46b84ebfe9..f3c277b67c 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/version/CoreVersion.java @@ -23,7 +23,7 @@ public class CoreVersion { public static final String NAME = "hugegraph-core"; - public static final String DEFAULT_VERSION = "1.5.0"; + public static final String DEFAULT_VERSION = "1.2.0"; /** * The second parameter of Version.of() is for IDE running without JAR */ diff --git a/hugegraph-server/hugegraph-dist/pom.xml b/hugegraph-server/hugegraph-dist/pom.xml index 9a58ac767f..cdec80950b 100644 --- a/hugegraph-server/hugegraph-dist/pom.xml +++ b/hugegraph-server/hugegraph-dist/pom.xml @@ -293,6 +293,22 @@ + + + + + + + + + + + diff --git a/hugegraph-server/pom.xml b/hugegraph-server/pom.xml index 2d5a51cb99..e5f09d22eb 100644 --- a/hugegraph-server/pom.xml +++ b/hugegraph-server/pom.xml @@ -310,6 +310,17 @@ ${final.name} + + + + ../${project.basedir} + + *.tar.gz + + + + ../${final.name} + diff --git a/hugegraph-store/README.md b/hugegraph-store/README.md index e69de29bb2..bef8b53c8a 100644 --- a/hugegraph-store/README.md +++ b/hugegraph-store/README.md @@ -0,0 +1,5 @@ +# HugeGraph Store + +HugeGraph Store is a new built-in storage backend, which uses RocksDB as the distributed backend storage engine. + +> Note: Currently, the contents of this folder are empty. Starting from revision 1.5.0, the code of HugeGraph Store will be adapted to this location (WIP). diff --git a/pom.xml b/pom.xml index f5e44e42dd..6917da79f6 100644 --- a/pom.xml +++ b/pom.xml @@ -89,7 +89,7 @@ - 1.5.0 + 1.2.0 From bd5d68f0f2cca3470473453e7d606472fb77e0c6 Mon Sep 17 00:00:00 2001 From: imbajin Date: Mon, 11 Dec 2023 22:21:08 +0800 Subject: [PATCH 6/9] refact(rocksdb): clean & reformat some code (#2200) * chore: merge master to clean-rocksdb for synchronization (#2383) --------- Co-authored-by: V_Galaxy --- .../hugegraph/backend/store/BackendTable.java | 14 +- .../backend/store/rocksdb/OpenedRocksDB.java | 20 +- .../store/rocksdb/RocksDBIngester.java | 14 +- .../store/rocksdb/RocksDBIteratorPool.java | 20 +- .../backend/store/rocksdb/RocksDBMetrics.java | 79 ++---- .../backend/store/rocksdb/RocksDBOptions.java | 11 +- .../store/rocksdb/RocksDBSessions.java | 25 +- .../store/rocksdb/RocksDBStdSessions.java | 245 +++++++----------- .../backend/store/rocksdb/RocksDBStore.java | 113 +++----- .../backend/store/rocksdb/RocksDBTable.java | 21 +- .../backend/store/rocksdb/RocksDBTables.java | 14 +- .../store/rocksdbsst/RocksDBSstSessions.java | 52 ++-- .../store/rocksdbsst/RocksDBSstStore.java | 33 +-- .../unit/rocksdb/BaseRocksDBUnitTest.java | 17 +- .../unit/rocksdb/RocksDBCountersTest.java | 9 +- .../unit/rocksdb/RocksDBPerfTest.java | 34 ++- .../unit/rocksdb/RocksDBSessionTest.java | 61 +++-- .../unit/rocksdb/RocksDBSessionsTest.java | 13 +- .../serializer/BinaryBackendEntryTest.java | 4 +- .../BinaryScatterSerializerTest.java | 8 +- .../unit/serializer/BinarySerializerTest.java | 5 +- 21 files changed, 320 insertions(+), 492 deletions(-) diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/store/BackendTable.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/store/BackendTable.java index 62d14782df..505739aefa 100644 --- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/store/BackendTable.java +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/store/BackendTable.java @@ -33,6 +33,7 @@ import org.apache.hugegraph.util.E; import org.apache.hugegraph.util.NumericUtil; import org.apache.hugegraph.util.StringEncoding; + import com.google.common.collect.ImmutableList; public abstract class BackendTable { @@ -91,7 +92,8 @@ public void updateIfAbsent(Session session, Entry entry) { } /** - * Mapping query-type to table-type + * Mapping query-type to table-type + * * @param query origin query * @return corresponding table type */ @@ -231,12 +233,11 @@ protected long maxKey() { public static class Range { - private byte[] startKey; - private byte[] endKey; + private final byte[] startKey; + private final byte[] endKey; public Range(byte[] startKey, byte[] endKey) { - this.startKey = Arrays.equals(EMPTY, startKey) ? - START_BYTES : startKey; + this.startKey = Arrays.equals(EMPTY, startKey) ? START_BYTES : startKey; this.endKey = Arrays.equals(EMPTY, endKey) ? END_BYTES : endKey; } @@ -361,8 +362,7 @@ public static byte[] increase(byte[] array) { private static byte[] align(byte[] array, int length) { int len = array.length; E.checkArgument(len <= length, - "The length of array '%s' exceed " + - "align length '%s'", len, length); + "The length of array '%s' exceed align length '%s'", len, length); byte[] target = new byte[length]; System.arraycopy(array, 0, target, length - len, len); return target; diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/OpenedRocksDB.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/OpenedRocksDB.java index 91e02878aa..c62ab12115 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/OpenedRocksDB.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/OpenedRocksDB.java @@ -27,17 +27,16 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.io.FileUtils; +import org.apache.hugegraph.backend.BackendException; +import org.apache.hugegraph.backend.store.rocksdb.RocksDBIteratorPool.ReusedRocksIterator; +import org.apache.hugegraph.util.E; +import org.apache.hugegraph.util.Log; import org.rocksdb.Checkpoint; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.RocksDB; import org.rocksdb.SstFileManager; import org.slf4j.Logger; -import org.apache.hugegraph.backend.BackendException; -import org.apache.hugegraph.backend.store.rocksdb.RocksDBIteratorPool.ReusedRocksIterator; -import org.apache.hugegraph.util.E; -import org.apache.hugegraph.util.Log; - public class OpenedRocksDB implements AutoCloseable { private static final Logger LOG = Log.logger(OpenedRocksDB.class); @@ -118,8 +117,7 @@ public void createCheckpoint(String targetPath) { tempFile, snapshotFile)); } } catch (Exception e) { - throw new BackendException("Failed to create checkpoint at path %s", - e, targetPath); + throw new BackendException("Failed to create checkpoint at path %s", e, targetPath); } } @@ -137,8 +135,7 @@ public CFHandle(RocksDB rocksdb, ColumnFamilyHandle handle) { } public synchronized ColumnFamilyHandle get() { - E.checkState(this.handle.isOwningHandle(), - "It seems CF has been closed"); + E.checkState(this.handle.isOwningHandle(), "It seems CF has been closed"); assert this.refs.get() >= 1; return this.handle; } @@ -163,7 +160,7 @@ public void close() { public synchronized ColumnFamilyHandle waitForDrop() { assert this.refs.get() >= 1; - // When entering this method, the refs won't increase any more + // When entering this method, the refs won't increase anymore final long timeout = TimeUnit.MINUTES.toMillis(30L); final long unit = 100L; for (long i = 1; this.refs.get() > 1; i++) { @@ -173,8 +170,7 @@ public synchronized ColumnFamilyHandle waitForDrop() { // 30s rest api timeout may cause InterruptedException } if (i * unit > timeout) { - throw new BackendException("Timeout after %sms to drop CF", - timeout); + throw new BackendException("Timeout after %sms to drop CF", timeout); } } assert this.refs.get() == 1; diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIngester.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIngester.java index ab89e19efc..fa30a389b5 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIngester.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIngester.java @@ -27,15 +27,14 @@ import java.util.ArrayList; import java.util.List; +import org.apache.hugegraph.backend.BackendException; +import org.apache.hugegraph.util.Log; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.IngestExternalFileOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.slf4j.Logger; -import org.apache.hugegraph.backend.BackendException; -import org.apache.hugegraph.util.Log; - public class RocksDBIngester { public static final String SST = ".sst"; @@ -52,8 +51,7 @@ public RocksDBIngester(RocksDB rocksdb) { this.options.setMoveFiles(true); } - public List ingest(Path path, ColumnFamilyHandle cf) - throws RocksDBException { + public List ingest(Path path, ColumnFamilyHandle cf) throws RocksDBException { SuffixFileVisitor visitor = new SuffixFileVisitor(SST); try { Files.walkFileTree(path, visitor); @@ -74,10 +72,8 @@ public List ingest(Path path, ColumnFamilyHandle cf) return ssts; } - public void ingest(ColumnFamilyHandle cf, List ssts) - throws RocksDBException { - LOG.info("Ingest sst files to CF '{}': {}", - RocksDBStdSessions.decode(cf.getName()), ssts); + public void ingest(ColumnFamilyHandle cf, List ssts) throws RocksDBException { + LOG.info("Ingest sst files to CF '{}': {}", RocksDBStdSessions.decode(cf.getName()), ssts); if (!ssts.isEmpty()) { this.rocksdb.ingestExternalFile(cf, ssts, this.options); } diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIteratorPool.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIteratorPool.java index 7aad1407ef..b4c6d3e2c1 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIteratorPool.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBIteratorPool.java @@ -20,17 +20,16 @@ import java.util.Queue; import java.util.concurrent.ArrayBlockingQueue; +import org.apache.hugegraph.backend.BackendException; +import org.apache.hugegraph.config.CoreOptions; +import org.apache.hugegraph.util.Log; +import org.apache.hugegraph.util.StringEncoding; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.rocksdb.RocksIterator; import org.slf4j.Logger; -import org.apache.hugegraph.backend.BackendException; -import org.apache.hugegraph.config.CoreOptions; -import org.apache.hugegraph.util.Log; -import org.apache.hugegraph.util.StringEncoding; - public final class RocksDBIteratorPool implements AutoCloseable { private static final Logger LOG = Log.logger(RocksDBIteratorPool.class); @@ -63,9 +62,8 @@ public ReusedRocksIterator newIterator() { @Override public void close() { - LOG.debug("Close IteratorPool with pool size {} ({})", - this.pool.size(), this); - for (RocksIterator iter; (iter = this.pool.poll()) != null;) { + LOG.debug("Close IteratorPool with pool size {} ({})", this.pool.size(), this); + for (RocksIterator iter; (iter = this.pool.poll()) != null; ) { this.closeIterator(iter); } assert this.pool.isEmpty(); @@ -149,13 +147,13 @@ private void closeIterator(RocksIterator iter) { protected final class ReusedRocksIterator { - private static final boolean EREUSING_ENABLED = false; + private static final boolean REUSING_ENABLED = false; private final RocksIterator iterator; private boolean closed; public ReusedRocksIterator() { this.closed = false; - if (EREUSING_ENABLED) { + if (REUSING_ENABLED) { this.iterator = allocIterator(); } else { this.iterator = createIterator(); @@ -173,7 +171,7 @@ public void close() { } this.closed = true; - if (EREUSING_ENABLED) { + if (REUSING_ENABLED) { releaseIterator(this.iterator); } else { closeIterator(this.iterator); diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBMetrics.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBMetrics.java index 61462d6f80..6547eaf76c 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBMetrics.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBMetrics.java @@ -24,6 +24,7 @@ import org.apache.hugegraph.util.Bytes; import org.apache.hugegraph.util.InsertionOrderUtil; import org.apache.hugegraph.util.UnitUtil; + import com.google.common.collect.ImmutableMap; public class RocksDBMetrics implements BackendMetrics { @@ -32,61 +33,37 @@ public class RocksDBMetrics implements BackendMetrics { private static final String PREFIX = "rocksdb."; // memory - private static final String BLOCK_CACHE = PREFIX + - "block-cache-usage"; - private static final String BLOCK_CACHE_PINNED = PREFIX + - "block-cache-pinned-usage"; - private static final String BLOCK_CACHE_CAPACITY = PREFIX + - "block-cache-capacity"; - private static final String INDEX_FILTER = PREFIX + - "estimate-table-readers-mem"; - private static final String ALL_MEM_TABLE = PREFIX + - "size-all-mem-tables"; - private static final String CUR_MEM_TABLE = PREFIX + - "cur-size-all-mem-tables"; + private static final String BLOCK_CACHE = PREFIX + "block-cache-usage"; + private static final String BLOCK_CACHE_PINNED = PREFIX + "block-cache-pinned-usage"; + private static final String BLOCK_CACHE_CAPACITY = PREFIX + "block-cache-capacity"; + private static final String INDEX_FILTER = PREFIX + "estimate-table-readers-mem"; + private static final String ALL_MEM_TABLE = PREFIX + "size-all-mem-tables"; + private static final String CUR_MEM_TABLE = PREFIX + "cur-size-all-mem-tables"; // disk - private static final String DISK_USAGE = PREFIX + - "disk-usage"; - private static final String LIVE_DATA_SIZE = PREFIX + - "estimate-live-data-size"; - private static final String SST_FILE_SIZE = PREFIX + - "total-sst-files-size"; - private static final String LIVE_SST_FILE_SIZE = PREFIX + - "live-sst-files-size"; + private static final String DISK_USAGE = PREFIX + "disk-usage"; + private static final String LIVE_DATA_SIZE = PREFIX + "estimate-live-data-size"; + private static final String SST_FILE_SIZE = PREFIX + "total-sst-files-size"; + private static final String LIVE_SST_FILE_SIZE = PREFIX + "live-sst-files-size"; private static final String PENDING_COMPACTION_BYTES = PREFIX + - "estimate-pending-compaction-bytes"; + "estimate-pending-compaction-bytes"; // count/number - private static final String NUM_KEYS = PREFIX + - "estimate-num-keys"; - private static final String NUM_KEYS_MEM_TABLE = PREFIX + - "num-entries-active-mem-table"; - private static final String NUM_KEYS_IMM_MEM_TABLE = PREFIX + - "num-entries-imm-mem-tables"; - private static final String NUM_DELETES_MEM_TABLE = PREFIX + - "num-deletes-active-mem-table"; - private static final String NUM_DELETES_IMM_MEM_TABLE = PREFIX + - "num-deletes-imm-mem-tables"; - - private static final String RUNNING_FLUSHS = PREFIX + - "num-running-flushes"; - private static final String MEM_TABLE_FLUSH_PENDINF = PREFIX + - "mem-table-flush-pending"; - private static final String RUNNING_COMPACTIONS = PREFIX + - "num-running-compactions"; - private static final String COMPACTION_PENDINF = PREFIX + - "compaction-pending"; - - private static final String NUM_IMM_MEM_TABLE = PREFIX + - "num-immutable-mem-table"; - private static final String NUM_SNAPSHOTS = PREFIX + - "num-snapshots"; - private static final String OLDEST_SNAPSHOT_TIME = PREFIX + - "oldest-snapshot-time"; - private static final String NUM_LIVE_VERSIONS = PREFIX + - "num-live-versions"; - private static final String SUPER_VERSION = PREFIX + - "current-super-version-number"; + private static final String NUM_KEYS = PREFIX + "estimate-num-keys"; + private static final String NUM_KEYS_MEM_TABLE = PREFIX + "num-entries-active-mem-table"; + private static final String NUM_KEYS_IMM_MEM_TABLE = PREFIX + "num-entries-imm-mem-tables"; + private static final String NUM_DELETES_MEM_TABLE = PREFIX + "num-deletes-active-mem-table"; + private static final String NUM_DELETES_IMM_MEM_TABLE = PREFIX + "num-deletes-imm-mem-tables"; + + private static final String RUNNING_FLUSHS = PREFIX + "num-running-flushes"; + private static final String MEM_TABLE_FLUSH_PENDINF = PREFIX + "mem-table-flush-pending"; + private static final String RUNNING_COMPACTIONS = PREFIX + "num-running-compactions"; + private static final String COMPACTION_PENDINF = PREFIX + "compaction-pending"; + + private static final String NUM_IMM_MEM_TABLE = PREFIX + "num-immutable-mem-table"; + private static final String NUM_SNAPSHOTS = PREFIX + "num-snapshots"; + private static final String OLDEST_SNAPSHOT_TIME = PREFIX + "oldest-snapshot-time"; + private static final String NUM_LIVE_VERSIONS = PREFIX + "num-live-versions"; + private static final String SUPER_VERSION = PREFIX + "current-super-version-number"; public static final String KEY_DISK_USAGE = DISK_USAGE; public static final String KEY_NUM_KEYS = NUM_KEYS; diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBOptions.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBOptions.java index a696b6cc3c..cb0b74a5d1 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBOptions.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBOptions.java @@ -23,17 +23,17 @@ import static org.apache.hugegraph.config.OptionChecker.rangeDouble; import static org.apache.hugegraph.config.OptionChecker.rangeInt; -import org.rocksdb.CompactionStyle; -import org.rocksdb.CompressionType; -import org.rocksdb.DataBlockIndexType; -import org.rocksdb.IndexType; - import org.apache.hugegraph.config.ConfigConvOption; import org.apache.hugegraph.config.ConfigListConvOption; import org.apache.hugegraph.config.ConfigListOption; import org.apache.hugegraph.config.ConfigOption; import org.apache.hugegraph.config.OptionHolder; import org.apache.hugegraph.util.Bytes; +import org.rocksdb.CompactionStyle; +import org.rocksdb.CompressionType; +import org.rocksdb.DataBlockIndexType; +import org.rocksdb.IndexType; + import com.google.common.collect.ImmutableList; public class RocksDBOptions extends OptionHolder { @@ -52,6 +52,7 @@ public static synchronized RocksDBOptions instance() { return instance; } + // TODO: the entire align style is wrong, change it to 4 space later public static final ConfigOption DATA_PATH = new ConfigOption<>( "rocksdb.data_path", diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBSessions.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBSessions.java index 8614d6b737..474f55db8f 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBSessions.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBSessions.java @@ -21,12 +21,11 @@ import java.util.Set; import org.apache.commons.lang3.tuple.Pair; -import org.rocksdb.RocksDBException; - import org.apache.hugegraph.backend.store.BackendEntry.BackendColumnIterator; import org.apache.hugegraph.backend.store.BackendSession.AbstractBackendSession; import org.apache.hugegraph.backend.store.BackendSessionPool; import org.apache.hugegraph.config.HugeConfig; +import org.rocksdb.RocksDBException; public abstract class RocksDBSessions extends BackendSessionPool { @@ -46,8 +45,7 @@ public RocksDBSessions(HugeConfig config, String database, String store) { public abstract void compactRange(); - public abstract RocksDBSessions copy(HugeConfig config, - String database, String store); + public abstract RocksDBSessions copy(HugeConfig config, String database, String store); public abstract void createSnapshot(String snapshotPath); @@ -55,8 +53,7 @@ public abstract RocksDBSessions copy(HugeConfig config, public abstract String buildSnapshotPath(String snapshotPrefix); - public abstract String hardLinkSnapshot(String snapshotPath) - throws RocksDBException; + public abstract String hardLinkSnapshot(String snapshotPath) throws RocksDBException; public abstract void reloadRocksDB() throws RocksDBException; @@ -105,22 +102,16 @@ public abstract void deleteRange(String table, public abstract byte[] get(String table, byte[] key); - public abstract BackendColumnIterator get(String table, - List keys); + public abstract BackendColumnIterator get(String table, List keys); public abstract BackendColumnIterator scan(String table); - public abstract BackendColumnIterator scan(String table, - byte[] prefix); + public abstract BackendColumnIterator scan(String table, byte[] prefix); - public abstract BackendColumnIterator scan(String table, - byte[] keyFrom, - byte[] keyTo, - int scanType); + public abstract BackendColumnIterator scan(String table, byte[] keyFrom, + byte[] keyTo, int scanType); - public BackendColumnIterator scan(String table, - byte[] keyFrom, - byte[] keyTo) { + public BackendColumnIterator scan(String table, byte[] keyFrom, byte[] keyTo) { return this.scan(table, keyFrom, keyTo, SCAN_LT_END); } diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index bcbe37b7c3..15f904d6e3 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -32,6 +32,18 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hugegraph.backend.BackendException; +import org.apache.hugegraph.backend.serializer.BinarySerializer; +import org.apache.hugegraph.backend.store.BackendEntry.BackendColumn; +import org.apache.hugegraph.backend.store.BackendEntry.BackendColumnIterator; +import org.apache.hugegraph.backend.store.BackendEntryIterator; +import org.apache.hugegraph.backend.store.rocksdb.RocksDBIteratorPool.ReusedRocksIterator; +import org.apache.hugegraph.config.CoreOptions; +import org.apache.hugegraph.config.HugeConfig; +import org.apache.hugegraph.util.Bytes; +import org.apache.hugegraph.util.E; +import org.apache.hugegraph.util.Log; +import org.apache.hugegraph.util.StringEncoding; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.BloomFilter; import org.rocksdb.ColumnFamilyDescriptor; @@ -57,18 +69,6 @@ import org.rocksdb.WriteOptions; import org.slf4j.Logger; -import org.apache.hugegraph.backend.BackendException; -import org.apache.hugegraph.backend.serializer.BinarySerializer; -import org.apache.hugegraph.backend.store.BackendEntry.BackendColumn; -import org.apache.hugegraph.backend.store.BackendEntry.BackendColumnIterator; -import org.apache.hugegraph.backend.store.BackendEntryIterator; -import org.apache.hugegraph.backend.store.rocksdb.RocksDBIteratorPool.ReusedRocksIterator; -import org.apache.hugegraph.config.CoreOptions; -import org.apache.hugegraph.config.HugeConfig; -import org.apache.hugegraph.util.Bytes; -import org.apache.hugegraph.util.E; -import org.apache.hugegraph.util.Log; -import org.apache.hugegraph.util.StringEncoding; import com.google.common.collect.ImmutableList; public class RocksDBStdSessions extends RocksDBSessions { @@ -83,14 +83,12 @@ public class RocksDBStdSessions extends RocksDBSessions { private final AtomicInteger refCount; public RocksDBStdSessions(HugeConfig config, String database, String store, - String dataPath, String walPath) - throws RocksDBException { + String dataPath, String walPath) throws RocksDBException { super(config, database, store); this.config = config; this.dataPath = dataPath; this.walPath = walPath; - this.rocksdb = RocksDBStdSessions.openRocksDB(config, dataPath, - walPath); + this.rocksdb = RocksDBStdSessions.openRocksDB(config, dataPath, walPath); this.refCount = new AtomicInteger(1); } @@ -101,8 +99,7 @@ public RocksDBStdSessions(HugeConfig config, String database, String store, this.config = config; this.dataPath = dataPath; this.walPath = walPath; - this.rocksdb = RocksDBStdSessions.openRocksDB(config, cfNames, - dataPath, walPath); + this.rocksdb = RocksDBStdSessions.openRocksDB(config, cfNames, dataPath, walPath); this.refCount = new AtomicInteger(1); this.ingestExternalFile(); @@ -166,8 +163,7 @@ public synchronized void createTable(String... tables) } @Override - public synchronized void dropTable(String... tables) - throws RocksDBException { + public synchronized void dropTable(String... tables) throws RocksDBException { this.checkValid(); /* @@ -210,10 +206,8 @@ public void reloadRocksDB() throws RocksDBException { if (this.rocksdb.isOwningHandle()) { this.rocksdb.close(); } - this.rocksdb = RocksDBStdSessions.openRocksDB(this.config, - ImmutableList.of(), - this.dataPath, - this.walPath); + this.rocksdb = RocksDBStdSessions.openRocksDB(this.config, ImmutableList.of(), + this.dataPath, this.walPath); } @Override @@ -252,8 +246,7 @@ public void compactRange() { } @Override - public RocksDBSessions copy(HugeConfig config, - String database, String store) { + public RocksDBSessions copy(HugeConfig config, String database, String store) { return new RocksDBStdSessions(config, database, store, this); } @@ -281,8 +274,7 @@ public void resumeSnapshot(String snapshotPath) { } // Move snapshot directory to origin data directory FileUtils.moveDirectory(snapshotDir, originDataDir); - LOG.info("Move snapshot directory {} to {}", - snapshotDir, originDataDir); + LOG.info("Move snapshot directory {} to {}", snapshotDir, originDataDir); // Reload rocksdb instance this.reloadRocksDB(); } catch (Exception e) { @@ -299,24 +291,20 @@ public String buildSnapshotPath(String snapshotPrefix) { // Like: rocksdb-data/* Path pureDataPath = parentParentPath.relativize(originDataPath.toAbsolutePath()); // Like: parent_path/snapshot_rocksdb-data/* - Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + - pureDataPath); + Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + pureDataPath); E.checkArgument(snapshotPath.toFile().exists(), - "The snapshot path '%s' doesn't exist", - snapshotPath); + "The snapshot path '%s' doesn't exist", snapshotPath); return snapshotPath.toString(); } @Override public String hardLinkSnapshot(String snapshotPath) throws RocksDBException { String snapshotLinkPath = this.dataPath + "_temp"; - try (OpenedRocksDB rocksdb = openRocksDB(this.config, - ImmutableList.of(), + try (OpenedRocksDB rocksdb = openRocksDB(this.config, ImmutableList.of(), snapshotPath, null)) { rocksdb.createCheckpoint(snapshotLinkPath); } - LOG.info("The snapshot {} has been hard linked to {}", - snapshotPath, snapshotLinkPath); + LOG.info("The snapshot {} has been hard linked to {}", snapshotPath, snapshotLinkPath); return snapshotLinkPath; } @@ -327,8 +315,7 @@ public final Session session() { @Override protected final Session newSession() { - E.checkState(this.rocksdb.isOwningHandle(), - "RocksDB has not been initialized"); + E.checkState(this.rocksdb.isOwningHandle(), "RocksDB has not been initialized"); return new StdSession(this.config()); } @@ -344,8 +331,7 @@ protected synchronized void doClose() { } private void checkValid() { - E.checkState(this.rocksdb.isOwningHandle(), - "It seems RocksDB has been closed"); + E.checkState(this.rocksdb.isOwningHandle(), "It seems RocksDB has been closed"); } private RocksDB rocksdb() { @@ -379,13 +365,11 @@ private void ingestExternalFile() throws RocksDBException { } } - private static OpenedRocksDB openRocksDB(HugeConfig config, - String dataPath, String walPath) - throws RocksDBException { + private static OpenedRocksDB openRocksDB(HugeConfig config, String dataPath, + String walPath) throws RocksDBException { // Init options Options options = new Options(); - RocksDBStdSessions.initOptions(config, options, options, - options, options); + RocksDBStdSessions.initOptions(config, options, options, options, options); options.setWalDir(walPath); SstFileManager sstFileManager = new SstFileManager(Env.getDefault()); options.setSstFileManager(sstFileManager); @@ -399,9 +383,8 @@ private static OpenedRocksDB openRocksDB(HugeConfig config, } private static OpenedRocksDB openRocksDB(HugeConfig config, - List cfNames, - String dataPath, String walPath) - throws RocksDBException { + List cfNames, String dataPath, + String walPath) throws RocksDBException { // Old CFs should always be opened Set mergedCFs = RocksDBStdSessions.mergeOldCFs(dataPath, cfNames); @@ -412,8 +395,7 @@ private static OpenedRocksDB openRocksDB(HugeConfig config, for (String cf : cfs) { ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor(encode(cf)); ColumnFamilyOptions options = cfd.getOptions(); - RocksDBStdSessions.initOptions(config, null, null, - options, options); + RocksDBStdSessions.initOptions(config, null, null, options, options); cfds.add(cfd); } @@ -440,8 +422,8 @@ private static OpenedRocksDB openRocksDB(HugeConfig config, return new OpenedRocksDB(rocksdb, cfHandles, sstFileManager); } - private static Set mergeOldCFs(String path, List cfNames) - throws RocksDBException { + private static Set mergeOldCFs(String path, + List cfNames) throws RocksDBException { Set cfs = listCFs(path); cfs.addAll(cfNames); return cfs; @@ -486,35 +468,28 @@ public static void initOptions(HugeConfig conf, db.setEnableWriteThreadAdaptiveYield(true); } db.setInfoLogLevel(InfoLogLevel.valueOf( - conf.get(RocksDBOptions.LOG_LEVEL) + "_LEVEL")); + conf.get(RocksDBOptions.LOG_LEVEL) + "_LEVEL")); - db.setMaxSubcompactions( - conf.get(RocksDBOptions.MAX_SUB_COMPACTIONS)); + db.setMaxSubcompactions(conf.get(RocksDBOptions.MAX_SUB_COMPACTIONS)); - db.setAllowMmapWrites( - conf.get(RocksDBOptions.ALLOW_MMAP_WRITES)); - db.setAllowMmapReads( - conf.get(RocksDBOptions.ALLOW_MMAP_READS)); + db.setAllowMmapWrites(conf.get(RocksDBOptions.ALLOW_MMAP_WRITES)); + db.setAllowMmapReads(conf.get(RocksDBOptions.ALLOW_MMAP_READS)); - db.setUseDirectReads( - conf.get(RocksDBOptions.USE_DIRECT_READS)); + db.setUseDirectReads(conf.get(RocksDBOptions.USE_DIRECT_READS)); db.setUseDirectIoForFlushAndCompaction( - conf.get(RocksDBOptions.USE_DIRECT_READS_WRITES_FC)); + conf.get(RocksDBOptions.USE_DIRECT_READS_WRITES_FC)); db.setUseFsync(conf.get(RocksDBOptions.USE_FSYNC)); db.setAtomicFlush(conf.get(RocksDBOptions.ATOMIC_FLUSH)); - db.setMaxManifestFileSize( - conf.get(RocksDBOptions.MAX_MANIFEST_FILE_SIZE)); + db.setMaxManifestFileSize(conf.get(RocksDBOptions.MAX_MANIFEST_FILE_SIZE)); - db.setSkipStatsUpdateOnDbOpen( - conf.get(RocksDBOptions.SKIP_STATS_UPDATE_ON_DB_OPEN)); + db.setSkipStatsUpdateOnDbOpen(conf.get(RocksDBOptions.SKIP_STATS_UPDATE_ON_DB_OPEN)); db.setSkipCheckingSstFileSizesOnDbOpen( - conf.get(RocksDBOptions.SKIP_CHECK_SIZE_ON_DB_OPEN)); + conf.get(RocksDBOptions.SKIP_CHECK_SIZE_ON_DB_OPEN)); - db.setMaxFileOpeningThreads( - conf.get(RocksDBOptions.MAX_FILE_OPENING_THREADS)); + db.setMaxFileOpeningThreads(conf.get(RocksDBOptions.MAX_FILE_OPENING_THREADS)); db.setDbWriteBufferSize(conf.get(RocksDBOptions.DB_MEMTABLE_SIZE)); @@ -535,8 +510,7 @@ public static void initOptions(HugeConfig conf, */ mdb.setMaxBackgroundJobs(conf.get(RocksDBOptions.MAX_BG_JOBS)); - mdb.setDelayedWriteRate( - conf.get(RocksDBOptions.DELAYED_WRITE_RATE)); + mdb.setDelayedWriteRate(conf.get(RocksDBOptions.DELAYED_WRITE_RATE)); mdb.setMaxOpenFiles(conf.get(RocksDBOptions.MAX_OPEN_FILES)); @@ -544,14 +518,12 @@ public static void initOptions(HugeConfig conf, mdb.setBytesPerSync(conf.get(RocksDBOptions.BYTES_PER_SYNC)); mdb.setWalBytesPerSync(conf.get(RocksDBOptions.WAL_BYTES_PER_SYNC)); - mdb.setStrictBytesPerSync( - conf.get(RocksDBOptions.STRICT_BYTES_PER_SYNC)); + mdb.setStrictBytesPerSync(conf.get(RocksDBOptions.STRICT_BYTES_PER_SYNC)); - mdb.setCompactionReadaheadSize( - conf.get(RocksDBOptions.COMPACTION_READAHEAD_SIZE)); + mdb.setCompactionReadaheadSize(conf.get(RocksDBOptions.COMPACTION_READAHEAD_SIZE)); - mdb.setDeleteObsoleteFilesPeriodMicros(1000000 * - conf.get(RocksDBOptions.DELETE_OBSOLETE_FILE_PERIOD)); + mdb.setDeleteObsoleteFilesPeriodMicros( + 1000000 * conf.get(RocksDBOptions.DELETE_OBSOLETE_FILE_PERIOD)); } if (cf != null) { @@ -562,38 +534,30 @@ public static void initOptions(HugeConfig conf, } int numLevels = conf.get(RocksDBOptions.NUM_LEVELS); - List compressions = conf.get( - RocksDBOptions.LEVELS_COMPRESSIONS); - E.checkArgument(compressions.isEmpty() || - compressions.size() == numLevels, + List compressions = conf.get(RocksDBOptions.LEVELS_COMPRESSIONS); + E.checkArgument(compressions.isEmpty() || compressions.size() == numLevels, "Elements number of '%s' must be 0 or " + "be the same as '%s', but got %s != %s", RocksDBOptions.LEVELS_COMPRESSIONS.name(), - RocksDBOptions.NUM_LEVELS.name(), - compressions.size(), numLevels); + RocksDBOptions.NUM_LEVELS.name(), compressions.size(), numLevels); cf.setNumLevels(numLevels); cf.setCompactionStyle(conf.get(RocksDBOptions.COMPACTION_STYLE)); - cf.setBottommostCompressionType( - conf.get(RocksDBOptions.BOTTOMMOST_COMPRESSION)); + cf.setBottommostCompressionType(conf.get(RocksDBOptions.BOTTOMMOST_COMPRESSION)); if (!compressions.isEmpty()) { cf.setCompressionPerLevel(compressions); } - cf.setMinWriteBufferNumberToMerge( - conf.get(RocksDBOptions.MIN_MEMTABLES_TO_MERGE)); + cf.setMinWriteBufferNumberToMerge(conf.get(RocksDBOptions.MIN_MEMTABLES_TO_MERGE)); cf.setMaxWriteBufferNumberToMaintain( - conf.get(RocksDBOptions.MAX_MEMTABLES_TO_MAINTAIN)); + conf.get(RocksDBOptions.MAX_MEMTABLES_TO_MAINTAIN)); - cf.setInplaceUpdateSupport( - conf.get(RocksDBOptions.MEMTABLE_INPLACE_UPDATE_SUPPORT)); + cf.setInplaceUpdateSupport(conf.get(RocksDBOptions.MEMTABLE_INPLACE_UPDATE_SUPPORT)); - cf.setLevelCompactionDynamicLevelBytes( - conf.get(RocksDBOptions.DYNAMIC_LEVEL_BYTES)); + cf.setLevelCompactionDynamicLevelBytes(conf.get(RocksDBOptions.DYNAMIC_LEVEL_BYTES)); - cf.setOptimizeFiltersForHits( - conf.get(RocksDBOptions.BLOOM_FILTERS_SKIP_LAST_LEVEL)); + cf.setOptimizeFiltersForHits(conf.get(RocksDBOptions.BLOOM_FILTERS_SKIP_LAST_LEVEL)); cf.setTableFormatConfig(initTableConfig(conf)); @@ -613,27 +577,22 @@ public static void initOptions(HugeConfig conf, mcf.setWriteBufferSize(conf.get(RocksDBOptions.MEMTABLE_SIZE)); mcf.setMaxWriteBufferNumber(conf.get(RocksDBOptions.MAX_MEMTABLES)); - mcf.setMaxBytesForLevelBase( - conf.get(RocksDBOptions.MAX_LEVEL1_BYTES)); - mcf.setMaxBytesForLevelMultiplier( - conf.get(RocksDBOptions.MAX_LEVEL_BYTES_MULTIPLIER)); + mcf.setMaxBytesForLevelBase(conf.get(RocksDBOptions.MAX_LEVEL1_BYTES)); + mcf.setMaxBytesForLevelMultiplier(conf.get(RocksDBOptions.MAX_LEVEL_BYTES_MULTIPLIER)); - mcf.setTargetFileSizeBase( - conf.get(RocksDBOptions.TARGET_FILE_SIZE_BASE)); - mcf.setTargetFileSizeMultiplier( - conf.get(RocksDBOptions.TARGET_FILE_SIZE_MULTIPLIER)); + mcf.setTargetFileSizeBase(conf.get(RocksDBOptions.TARGET_FILE_SIZE_BASE)); + mcf.setTargetFileSizeMultiplier(conf.get(RocksDBOptions.TARGET_FILE_SIZE_MULTIPLIER)); mcf.setLevel0FileNumCompactionTrigger( - conf.get(RocksDBOptions.LEVEL0_COMPACTION_TRIGGER)); + conf.get(RocksDBOptions.LEVEL0_COMPACTION_TRIGGER)); mcf.setLevel0SlowdownWritesTrigger( - conf.get(RocksDBOptions.LEVEL0_SLOWDOWN_WRITES_TRIGGER)); - mcf.setLevel0StopWritesTrigger( - conf.get(RocksDBOptions.LEVEL0_STOP_WRITES_TRIGGER)); + conf.get(RocksDBOptions.LEVEL0_SLOWDOWN_WRITES_TRIGGER)); + mcf.setLevel0StopWritesTrigger(conf.get(RocksDBOptions.LEVEL0_STOP_WRITES_TRIGGER)); mcf.setSoftPendingCompactionBytesLimit( - conf.get(RocksDBOptions.SOFT_PENDING_COMPACTION_LIMIT)); + conf.get(RocksDBOptions.SOFT_PENDING_COMPACTION_LIMIT)); mcf.setHardPendingCompactionBytesLimit( - conf.get(RocksDBOptions.HARD_PENDING_COMPACTION_LIMIT)); + conf.get(RocksDBOptions.HARD_PENDING_COMPACTION_LIMIT)); /* * TODO: also set memtable options: @@ -643,11 +602,10 @@ public static void initOptions(HugeConfig conf, * #diff-cde52d1fcbcce2bc6aae27838f1d3e7e9e469ccad8aaf8f2695f939e279d7501R369 */ mcf.setMemtablePrefixBloomSizeRatio( - conf.get(RocksDBOptions.MEMTABLE_BLOOM_SIZE_RATIO)); + conf.get(RocksDBOptions.MEMTABLE_BLOOM_SIZE_RATIO)); mcf.setMemtableWholeKeyFiltering( - conf.get(RocksDBOptions.MEMTABLE_BLOOM_WHOLE_KEY_FILTERING)); - mcf.setMemtableHugePageSize( - conf.get(RocksDBOptions.MEMTABL_BLOOM_HUGE_PAGE_SIZE)); + conf.get(RocksDBOptions.MEMTABLE_BLOOM_WHOLE_KEY_FILTERING)); + mcf.setMemtableHugePageSize(conf.get(RocksDBOptions.MEMTABL_BLOOM_HUGE_PAGE_SIZE)); boolean bulkload = conf.get(RocksDBOptions.BULKLOAD_MODE); if (bulkload) { @@ -671,8 +629,7 @@ public static void initOptions(HugeConfig conf, public static TableFormatConfig initTableConfig(HugeConfig conf) { BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); - tableConfig.setFormatVersion( - conf.get(RocksDBOptions.TABLE_FORMAT_VERSION)); + tableConfig.setFormatVersion(conf.get(RocksDBOptions.TABLE_FORMAT_VERSION)); /* * The index type used to lookup between data blocks: @@ -689,17 +646,14 @@ public static TableFormatConfig initTableConfig(HugeConfig conf) { * The search type of point lookup can be BinarySearch or HashSearch: * https://github.com/facebook/rocksdb/wiki/Data-Block-Hash-Index */ - tableConfig.setDataBlockIndexType( - conf.get(RocksDBOptions.DATA_BLOCK_SEARCH_TYPE)); + tableConfig.setDataBlockIndexType(conf.get(RocksDBOptions.DATA_BLOCK_SEARCH_TYPE)); tableConfig.setDataBlockHashTableUtilRatio( - conf.get(RocksDBOptions.DATA_BLOCK_HASH_TABLE_RATIO)); + conf.get(RocksDBOptions.DATA_BLOCK_HASH_TABLE_RATIO)); long blockSize = conf.get(RocksDBOptions.BLOCK_SIZE); tableConfig.setBlockSize(blockSize); - tableConfig.setBlockSizeDeviation( - conf.get(RocksDBOptions.BLOCK_SIZE_DEVIATION)); - tableConfig.setBlockRestartInterval( - conf.get(RocksDBOptions.BLOCK_RESTART_INTERVAL)); + tableConfig.setBlockSizeDeviation(conf.get(RocksDBOptions.BLOCK_SIZE_DEVIATION)); + tableConfig.setBlockRestartInterval(conf.get(RocksDBOptions.BLOCK_RESTART_INTERVAL)); // https://github.com/facebook/rocksdb/wiki/Block-Cache long cacheCapacity = conf.get(RocksDBOptions.BLOCK_CACHE_CAPACITY); @@ -715,16 +669,14 @@ public static TableFormatConfig initTableConfig(HugeConfig conf) { if (bitsPerKey >= 0) { // TODO: use space-saving RibbonFilterPolicy boolean blockBased = conf.get(RocksDBOptions.BLOOM_FILTER_MODE); - tableConfig.setFilterPolicy(new BloomFilter(bitsPerKey, - blockBased)); + tableConfig.setFilterPolicy(new BloomFilter(bitsPerKey, blockBased)); - tableConfig.setWholeKeyFiltering( - conf.get(RocksDBOptions.BLOOM_FILTER_WHOLE_KEY)); + tableConfig.setWholeKeyFiltering(conf.get(RocksDBOptions.BLOOM_FILTER_WHOLE_KEY)); tableConfig.setCacheIndexAndFilterBlocks( - conf.get(RocksDBOptions.CACHE_FILTER_AND_INDEX)); + conf.get(RocksDBOptions.CACHE_FILTER_AND_INDEX)); tableConfig.setPinL0FilterAndIndexBlocksInCache( - conf.get(RocksDBOptions.PIN_L0_INDEX_AND_FILTER)); + conf.get(RocksDBOptions.PIN_L0_INDEX_AND_FILTER)); // https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters if (conf.get(RocksDBOptions.PARTITION_FILTERS_INDEXES)) { @@ -734,7 +686,7 @@ public static TableFormatConfig initTableConfig(HugeConfig conf) { .setMetadataBlockSize(blockSize) .setCacheIndexAndFilterBlocksWithHighPriority(true); tableConfig.setPinTopLevelIndexAndFilter( - conf.get(RocksDBOptions.PIN_TOP_INDEX_AND_FILTER)); + conf.get(RocksDBOptions.PIN_TOP_INDEX_AND_FILTER)); } } @@ -898,7 +850,7 @@ public void put(String table, byte[] key, byte[] value) { /** * Merge a record to an existing key to a table * For more details about merge-operator: - * https://github.com/facebook/rocksdb/wiki/merge-operator + * ... */ @Override public void merge(String table, byte[] key, byte[] value) { @@ -950,8 +902,7 @@ public void deleteSingle(String table, byte[] key) { * Delete a record by key(or prefix with key) from a table */ @Override - public void deletePrefix(String table, byte[] key) { - byte[] keyFrom = key; + public void deletePrefix(String table, byte[] keyFrom) { byte[] keyTo = Arrays.copyOf(keyFrom, keyFrom.length); BinarySerializer.increaseOne(keyTo); try (OpenedRocksDB.CFHandle cf = cf(table)) { @@ -1044,8 +995,7 @@ public BackendColumnIterator scan(String table, byte[] prefix) { */ try (OpenedRocksDB.CFHandle cf = cf(table)) { ReusedRocksIterator iter = cf.newIterator(); - return new ScanIterator(table, iter, prefix, null, - SCAN_PREFIX_BEGIN); + return new ScanIterator(table, iter, prefix, null, SCAN_PREFIX_BEGIN); } } @@ -1076,8 +1026,7 @@ public BackendColumnIterator scan(String table, byte[] keyFrom, /** * A wrapper for RocksIterator that convert RocksDB results to std Iterator */ - private static class ScanIterator implements BackendColumnIterator, - Countable { + private static class ScanIterator implements BackendColumnIterator, Countable { private final String table; private final ReusedRocksIterator reusedIter; @@ -1164,14 +1113,12 @@ private boolean match(int expected) { @SuppressWarnings("unused") private void dump() { this.seek(); - LOG.info(">>>> scan from {}: {}{}", - this.table, - this.keyBegin == null ? "*" : StringEncoding.format(this.keyBegin), - this.iter.isValid() ? "" : " - No data"); + LOG.info(">>>> scan from {}: {}{}", this.table, + this.keyBegin == null ? "*" : StringEncoding.format(this.keyBegin), + this.iter.isValid() ? "" : " - No data"); for (; this.iter.isValid(); this.iter.next()) { - LOG.info("{}={}", - StringEncoding.format(this.iter.key()), - StringEncoding.format(this.iter.value())); + LOG.info("{}={}", StringEncoding.format(this.iter.key()), + StringEncoding.format(this.iter.value())); } } @@ -1202,7 +1149,7 @@ public boolean hasNext() { } private void seek() { - if (this.keyBegin == null || this.keyBegin.length <= 0) { + if (this.keyBegin == null || this.keyBegin.length == 0) { // Seek to the first if no `keyBegin` this.iter.seekToFirst(); } else { @@ -1216,8 +1163,7 @@ private void seek() { // Skip `keyBegin` if set SCAN_GT_BEGIN (key > 'xx') if (this.match(Session.SCAN_GT_BEGIN) && !this.match(Session.SCAN_GTE_BEGIN)) { - while (this.iter.isValid() && - Bytes.equals(this.iter.key(), this.keyBegin)) { + while (this.iter.isValid() && Bytes.equals(this.iter.key(), this.keyBegin)) { this.iter.next(); } } @@ -1254,10 +1200,8 @@ private boolean filter(byte[] key) { return Bytes.compare(key, this.keyEnd) < 0; } } else { - assert this.match(Session.SCAN_ANY) || - this.match(Session.SCAN_GT_BEGIN) || - this.match(Session.SCAN_GTE_BEGIN) : - "Unknow scan type"; + assert this.match(Session.SCAN_ANY) || this.match(Session.SCAN_GT_BEGIN) || + this.match(Session.SCAN_GTE_BEGIN) : "Unknown scan type"; return true; } } @@ -1270,8 +1214,7 @@ public BackendColumn next() { } } - BackendColumn col = BackendColumn.of(this.iter.key(), - this.iter.value()); + BackendColumn col = BackendColumn.of(this.iter.key(), this.iter.value()); this.iter.next(); this.matched = false; diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStore.java index 283baa622a..c9a27b7705 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -93,9 +93,10 @@ public abstract class RocksDBStore extends AbstractBackendStore(); this.olapTables = new HashMap<>(); - this.provider = provider; this.database = database; this.store = store; @@ -221,8 +221,7 @@ public synchronized void open(HugeConfig config) { } List> futures = new ArrayList<>(); - ExecutorService openPool = ExecutorUtil.newFixedThreadPool( - OPEN_POOL_THREADS, DB_OPEN); + ExecutorService openPool = ExecutorUtil.newFixedThreadPool(OPEN_POOL_THREADS, DB_OPEN); // Open base disk futures.add(openPool.submit(() -> { this.sessions = this.open(config, this.tableNames()); @@ -282,8 +281,7 @@ private void shutdownOpenPool(ExecutorService openPool) { Consumers.executeOncePerThread(openPool, OPEN_POOL_THREADS, this::closeSessions, DB_CLOSE_TIMEOUT); } catch (InterruptedException e) { - throw new BackendException("Failed to close session opened by " + - "open-pool"); + throw new BackendException("Failed to close session opened by open-pool"); } boolean terminated; @@ -292,8 +290,7 @@ private void shutdownOpenPool(ExecutorService openPool) { terminated = openPool.awaitTermination(DB_OPEN_TIMEOUT, TimeUnit.SECONDS); } catch (Throwable e) { - throw new BackendException( - "Failed to wait db-open thread pool shutdown", e); + throw new BackendException("Failed to wait db-open thread pool shutdown", e); } if (!terminated) { LOG.warn("Timeout when waiting db-open thread pool shutdown"); @@ -346,8 +343,7 @@ protected RocksDBSessions open(HugeConfig config, String dataPath, none = null; } try { - sessions = this.openSessionPool(config, dataPath, - walPath, none); + sessions = this.openSessionPool(config, dataPath, walPath, none); } catch (RocksDBException e1) { e = e1; } @@ -360,8 +356,7 @@ protected RocksDBSessions open(HugeConfig config, String dataPath, if (sessions == null) { // Error after trying other ways LOG.error("Failed to open RocksDB '{}'", dataPath, e); - throw new ConnectionException("Failed to open RocksDB '%s'", - e, dataPath); + throw new ConnectionException("Failed to open RocksDB '%s'", e, dataPath); } } @@ -377,11 +372,9 @@ protected RocksDBSessions open(HugeConfig config, String dataPath, protected RocksDBSessions openSessionPool(HugeConfig config, String dataPath, String walPath, - List tableNames) - throws RocksDBException { + List tableNames) throws RocksDBException { if (tableNames == null) { - return new RocksDBStdSessions(config, this.database, this.store, - dataPath, walPath); + return new RocksDBStdSessions(config, this.database, this.store, dataPath, walPath); } else { return new RocksDBStdSessions(config, this.database, this.store, dataPath, walPath, tableNames); @@ -404,8 +397,7 @@ protected Map tableDBMapping() { for (Entry e : this.tableDiskMapping.entrySet()) { HugeType type = e.getKey(); RocksDBSessions db = this.db(e.getValue()); - String key = type != HugeType.OLAP ? this.table(type).table() : - type.string(); + String key = type != HugeType.OLAP ? this.table(type).table() : type.string(); tableDBMap.put(key, db); } return tableDBMap; @@ -418,7 +410,6 @@ protected ReadWriteLock storeLock() { @Override public void close() { LOG.debug("Store close: {}", this.store); - this.checkOpened(); this.closeSessions(); } @@ -435,15 +426,13 @@ public void mutate(BackendMutation mutation) { readLock.lock(); try { this.checkOpened(); - if (LOG.isDebugEnabled()) { LOG.debug("Store {} mutation: {}", this.store, mutation); } for (HugeType type : mutation.types()) { RocksDBSessions.Session session = this.session(type); - for (Iterator it = mutation.mutation(type); - it.hasNext();) { + for (Iterator it = mutation.mutation(type); it.hasNext(); ) { this.mutate(session, it.next()); } } @@ -454,8 +443,8 @@ public void mutate(BackendMutation mutation) { private void mutate(RocksDBSessions.Session session, BackendAction item) { BackendEntry entry = item.entry(); - RocksDBTable table; + if (!entry.olap()) { // Oltp table table = this.table(entry.type()); @@ -469,6 +458,7 @@ private void mutate(RocksDBSessions.Session session, BackendAction item) { } session = this.session(HugeType.OLAP); } + switch (item.action()) { case INSERT: table.insert(session, entry); @@ -489,8 +479,8 @@ private void mutate(RocksDBSessions.Session session, BackendAction item) { table.updateIfAbsent(session, entry); break; default: - throw new AssertionError(String.format( - "Unsupported mutate action: %s", item.action())); + throw new AssertionError(String.format("Unsupported mutate action: %s", + item.action())); } } @@ -498,9 +488,9 @@ private void mutate(RocksDBSessions.Session session, BackendAction item) { public Iterator query(Query query) { Lock readLock = this.storeLock.readLock(); readLock.lock(); + try { this.checkOpened(); - HugeType tableType = RocksDBTable.tableType(query); RocksDBTable table; RocksDBSessions.Session session; @@ -522,8 +512,7 @@ public Iterator query(Query query) { table = this.table(this.olapTableName(pk)); iterators.add(table.query(this.session(HugeType.OLAP), q)); } - entries = new MergeIterator<>(entries, iterators, - BackendEntry::mergeable); + entries = new MergeIterator<>(entries, iterators, BackendEntry::mergeable); } return entries; } finally { @@ -537,7 +526,6 @@ public Number queryNumber(Query query) { readLock.lock(); try { this.checkOpened(); - HugeType tableType = RocksDBTable.tableType(query); RocksDBTable table = this.table(tableType); return table.queryNumber(this.session(tableType), query); @@ -552,10 +540,8 @@ public synchronized void init() { writeLock.lock(); try { this.checkDbOpened(); - // Create tables with main disk - this.createTable(this.sessions, - this.tableNames().toArray(new String[0])); + this.createTable(this.sessions, this.tableNames().toArray(new String[0])); // Create table with optimized disk Map tableDBMap = this.tableDBMapping(); @@ -590,10 +576,8 @@ public synchronized void clear(boolean clearSpace) { writeLock.lock(); try { this.checkDbOpened(); - // Drop tables with main disk - this.dropTable(this.sessions, - this.tableNames().toArray(new String[0])); + this.dropTable(this.sessions, this.tableNames().toArray(new String[0])); // Drop tables with optimized disk Map tableDBMap = this.tableDBMapping(); @@ -630,10 +614,10 @@ protected void dropTable(RocksDBSessions db, String... tables) { @Override public boolean initialized() { this.checkDbOpened(); - if (!this.opened()) { return false; } + for (String table : this.tableNames()) { if (!this.sessions.existsTable(table)) { return false; @@ -726,7 +710,7 @@ public Map createSnapshot(String snapshotPrefix) { readLock.lock(); try { Map uniqueSnapshotDirMaps = new HashMap<>(); - // Every rocksdb instance should create an snapshot + // Every rocksdb instance should create a snapshot for (Map.Entry entry : this.dbs.entrySet()) { // Like: parent_path/rocksdb-data/*, * maybe g,m,s Path originDataPath = Paths.get(entry.getKey()).toAbsolutePath(); @@ -743,8 +727,7 @@ public Map createSnapshot(String snapshotPrefix) { String snapshotDir = snapshotPath.toAbsolutePath().getParent().toString(); // Find correspond data HugeType key - String diskTableKey = this.findDiskTableKeyByPath( - entry.getKey()); + String diskTableKey = this.findDiskTableKeyByPath(entry.getKey()); uniqueSnapshotDirMaps.put(snapshotDir, diskTableKey); } LOG.info("The store '{}' create snapshot successfully", this); @@ -775,7 +758,7 @@ public void resumeSnapshot(String snapshotPrefix, boolean deleteSnapshot) { } for (Map.Entry entry : - snapshotPaths.entrySet()) { + snapshotPaths.entrySet()) { String snapshotPath = entry.getKey(); RocksDBSessions sessions = entry.getValue(); sessions.resumeSnapshot(snapshotPath); @@ -819,8 +802,7 @@ private List session() { } private void closeSessions() { - Iterator> iter = this.dbs.entrySet() - .iterator(); + Iterator> iter = this.dbs.entrySet().iterator(); while (iter.hasNext()) { Map.Entry entry = iter.next(); RocksDBSessions sessions = entry.getValue(); @@ -835,23 +817,20 @@ private Collection sessions() { return this.dbs.values(); } - private void parseTableDiskMapping(Map disks, - String dataPath) { + private void parseTableDiskMapping(Map disks, String dataPath) { this.tableDiskMapping.clear(); for (Map.Entry disk : disks.entrySet()) { // The format of `disk` like: `graph/vertex: /path/to/disk1` String name = disk.getKey(); String path = disk.getValue(); - E.checkArgument(!dataPath.equals(path), "Invalid disk path" + - "(can't be the same as data_path): '%s'", path); + E.checkArgument(!dataPath.equals(path), + "Invalid disk path (can't be the same as data_path): '%s'", path); E.checkArgument(!name.isEmpty() && !path.isEmpty(), - "Invalid disk format: '%s', expect `NAME:PATH`", - disk); + "Invalid disk format: '%s', expect `NAME:PATH`", disk); String[] pair = name.split("/", 2); E.checkArgument(pair.length == 2, - "Invalid disk key format: '%s', " + - "expect `STORE/TABLE`", name); + "Invalid disk key format: '%s', expect `STORE/TABLE`", name); String store = pair[0].trim(); HugeType table = HugeType.valueOf(pair[1].trim().toUpperCase()); if (this.store.equals(store)) { @@ -948,14 +927,10 @@ public RocksDBSchemaStore(BackendStoreProvider provider, this.counters = new RocksDBTables.Counters(database); - registerTableManager(HugeType.VERTEX_LABEL, - new RocksDBTables.VertexLabel(database)); - registerTableManager(HugeType.EDGE_LABEL, - new RocksDBTables.EdgeLabel(database)); - registerTableManager(HugeType.PROPERTY_KEY, - new RocksDBTables.PropertyKey(database)); - registerTableManager(HugeType.INDEX_LABEL, - new RocksDBTables.IndexLabel(database)); + registerTableManager(HugeType.VERTEX_LABEL, new RocksDBTables.VertexLabel(database)); + registerTableManager(HugeType.EDGE_LABEL, new RocksDBTables.EdgeLabel(database)); + registerTableManager(HugeType.PROPERTY_KEY, new RocksDBTables.PropertyKey(database)); + registerTableManager(HugeType.INDEX_LABEL, new RocksDBTables.IndexLabel(database)); registerTableManager(HugeType.SECONDARY_INDEX, new RocksDBTables.SecondaryIndex(database)); } @@ -1005,13 +980,10 @@ public RocksDBGraphStore(BackendStoreProvider provider, String database, String store) { super(provider, database, store); - registerTableManager(HugeType.VERTEX, - new RocksDBTables.Vertex(database)); + registerTableManager(HugeType.VERTEX, new RocksDBTables.Vertex(database)); - registerTableManager(HugeType.EDGE_OUT, - RocksDBTables.Edge.out(database)); - registerTableManager(HugeType.EDGE_IN, - RocksDBTables.Edge.in(database)); + registerTableManager(HugeType.EDGE_OUT, RocksDBTables.Edge.out(database)); + registerTableManager(HugeType.EDGE_IN, RocksDBTables.Edge.in(database)); registerTableManager(HugeType.SECONDARY_INDEX, new RocksDBTables.SecondaryIndex(database)); @@ -1053,20 +1025,17 @@ public boolean isSchemaStore() { @Override public Id nextId(HugeType type) { - throw new UnsupportedOperationException( - "RocksDBGraphStore.nextId()"); + throw new UnsupportedOperationException("RocksDBGraphStore.nextId()"); } @Override public void increaseCounter(HugeType type, long num) { - throw new UnsupportedOperationException( - "RocksDBGraphStore.increaseCounter()"); + throw new UnsupportedOperationException("RocksDBGraphStore.increaseCounter()"); } @Override public long getCounter(HugeType type) { - throw new UnsupportedOperationException( - "RocksDBGraphStore.getCounter()"); + throw new UnsupportedOperationException("RocksDBGraphStore.getCounter()"); } /** @@ -1117,10 +1086,8 @@ public static class RocksDBSystemStore extends RocksDBGraphStore { private final RocksDBTables.Meta meta; - public RocksDBSystemStore(BackendStoreProvider provider, - String database, String store) { + public RocksDBSystemStore(BackendStoreProvider provider, String database, String store) { super(provider, database, store); - this.meta = new RocksDBTables.Meta(database); } diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTable.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTable.java index 7a5af5f1ae..ec2959d32c 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTable.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTable.java @@ -25,9 +25,6 @@ import java.util.Set; import org.apache.commons.lang3.tuple.Pair; -import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; -import org.slf4j.Logger; - import org.apache.hugegraph.backend.id.Id; import org.apache.hugegraph.backend.page.PageState; import org.apache.hugegraph.backend.query.Aggregate; @@ -52,6 +49,8 @@ import org.apache.hugegraph.util.E; import org.apache.hugegraph.util.Log; import org.apache.hugegraph.util.StringEncoding; +import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; +import org.slf4j.Logger; public class RocksDBTable extends BackendTable { @@ -67,8 +66,7 @@ public RocksDBTable(String database, String table) { @Override protected void registerMetaHandlers() { this.registerMetaHandler("splits", (session, meta, args) -> { - E.checkArgument(args.length == 1, - "The args count of %s must be 1", meta); + E.checkArgument(args.length == 1, "The args count of %s must be 1", meta); long splitSize = (long) args[0]; return this.shardSplitter.getSplits(session, splitSize); }); @@ -203,7 +201,7 @@ protected BackendColumnIterator queryByIds(RocksDBSessions.Session session, // NOTE: this will lead to lazy create rocksdb iterator return BackendColumnIterator.wrap(new FlatMapperIterator<>( - ids.iterator(), id -> this.queryById(session, id) + ids.iterator(), id -> this.queryById(session, id) )); } @@ -233,8 +231,7 @@ protected BackendColumnIterator queryByPrefix(RocksDBSessions.Session session, int type = query.inclusiveStart() ? RocksDBSessions.Session.SCAN_GTE_BEGIN : RocksDBSessions.Session.SCAN_GT_BEGIN; type |= RocksDBSessions.Session.SCAN_PREFIX_END; - return session.scan(this.table(), query.start().asBytes(), - query.prefix().asBytes(), type); + return session.scan(this.table(), query.start().asBytes(), query.prefix().asBytes(), type); } protected BackendColumnIterator queryByRange(RocksDBSessions.Session session, @@ -268,8 +265,7 @@ protected BackendColumnIterator queryByRange(RocksDBSessions.Session session, Sh byte[] end = this.shardSplitter.position(shard.end()); if (page != null && !page.isEmpty()) { byte[] position = PageState.fromString(page).position(); - E.checkArgument(start == null || - Bytes.compare(position, start) >= 0, + E.checkArgument(start == null || Bytes.compare(position, start) >= 0, "Invalid page out of lower bound"); start = position; } @@ -310,7 +306,6 @@ private static class RocksDBShardSplitter extends ShardSplitter getSplits(RocksDBSessions.Session session, long splitSize) { count = 1; } - Range range = new Range(keyRange.getLeft(), - Range.increase(keyRange.getRight())); + Range range = new Range(keyRange.getLeft(), Range.increase(keyRange.getRight())); List splits = new ArrayList<>((int) count); splits.addAll(range.splitEven((int) count)); return splits; @@ -359,6 +353,7 @@ public long estimateNumKeys(RocksDBSessions.Session session) { @Override public byte[] position(String position) { + // TODO: START & END is same & be empty now? remove one? if (START.equals(position) || END.equals(position)) { return null; } diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTables.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTables.java index 06c2d91a1a..dad0545ade 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTables.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdb/RocksDBTables.java @@ -120,8 +120,7 @@ public void delete(RocksDBSessions.Session session, BackendEntry entry) { * `scanPrefix + delete`: session.delete(scanPrefix(prefix)) */ byte[] prefix = entry.id().asBytes(); - try (BackendColumnIterator results = session.scan(this.table(), - prefix)) { + try (BackendColumnIterator results = session.scan(this.table(), prefix)) { while (results.hasNext()) { byte[] column = results.next().name; session.delete(this.table(), column); @@ -218,6 +217,7 @@ public IndexTable(String database, String table) { } @Override + // TODO: why this method is same as super.eliminate() in RocksDBTable, del it? public void eliminate(RocksDBSessions.Session session, BackendEntry entry) { assert entry.columns().size() == 1; super.delete(session, entry); @@ -291,10 +291,8 @@ public RangeIndex(String database, String table) { protected BackendColumnIterator queryByCond(RocksDBSessions.Session session, ConditionQuery query) { assert query.conditionsSize() > 0; - List conds = query.syspropConditions(HugeKeys.ID); - E.checkArgument(!conds.isEmpty(), - "Please specify the index conditions"); + E.checkArgument(!conds.isEmpty(), "Please specify the index conditions"); Id prefix = null; Id min = null; @@ -323,8 +321,7 @@ protected BackendColumnIterator queryByCond(RocksDBSessions.Session session, max = (Id) r.value(); break; default: - E.checkArgument(false, "Unsupported relation '%s'", - r.relation()); + E.checkArgument(false, "Unsupported relation '%s'", r.relation()); } } @@ -340,7 +337,8 @@ protected BackendColumnIterator queryByCond(RocksDBSessions.Session session, RocksDBSessions.Session.SCAN_PREFIX_END); } else { byte[] end = max.asBytes(); - int type = maxEq ? RocksDBSessions.Session.SCAN_LTE_END : RocksDBSessions.Session.SCAN_LT_END; + int type = maxEq ? RocksDBSessions.Session.SCAN_LTE_END + : RocksDBSessions.Session.SCAN_LT_END; return session.scan(this.table(), begin, end, type); } } diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java index 3d2b7f867a..d7ce2db878 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java @@ -31,11 +31,6 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.tuple.Pair; -import org.rocksdb.EnvOptions; -import org.rocksdb.Options; -import org.rocksdb.RocksDBException; -import org.rocksdb.SstFileWriter; - import org.apache.hugegraph.backend.BackendException; import org.apache.hugegraph.backend.store.BackendEntry.BackendColumnIterator; import org.apache.hugegraph.backend.store.rocksdb.RocksDBIngester; @@ -44,14 +39,17 @@ import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.exception.NotSupportException; import org.apache.hugegraph.util.E; +import org.rocksdb.EnvOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDBException; +import org.rocksdb.SstFileWriter; public class RocksDBSstSessions extends RocksDBSessions { private final String dataPath; private final Map tables; - public RocksDBSstSessions(HugeConfig config, String database, String store, - String dataPath) { + public RocksDBSstSessions(HugeConfig config, String database, String store, String dataPath) { super(config, database, store); this.dataPath = dataPath; @@ -63,8 +61,7 @@ public RocksDBSstSessions(HugeConfig config, String database, String store, } } - public RocksDBSstSessions(HugeConfig config, String dataPath, - String database, String store, + public RocksDBSstSessions(HugeConfig config, String dataPath, String database, String store, List tableNames) throws RocksDBException { this(config, dataPath, database, store); for (String table : tableNames) { @@ -96,8 +93,7 @@ public Set openedTables() { } @Override - public synchronized void createTable(String... tables) - throws RocksDBException { + public synchronized void createTable(String... tables) throws RocksDBException { for (String table : tables) { this.createTable(table); } @@ -105,8 +101,7 @@ public synchronized void createTable(String... tables) private void createTable(String table) throws RocksDBException { String number = String.format("%04d", 1); - Path sstFile = Paths.get(this.dataPath, table, - number + RocksDBIngester.SST); + Path sstFile = Paths.get(this.dataPath, table, number + RocksDBIngester.SST); try { FileUtils.forceMkdir(sstFile.toAbsolutePath().getParent().toFile()); } catch (IOException e) { @@ -116,8 +111,7 @@ private void createTable(String table) throws RocksDBException { EnvOptions env = new EnvOptions(); Options options = new Options(); - RocksDBStdSessions.initOptions(this.config(), options, options, - options, options); + RocksDBStdSessions.initOptions(this.config(), options, options, options, options); // NOTE: unset merge op due to SIGSEGV when cf.setMergeOperatorName() options.setMergeOperatorName("not-exist-merge-op"); SstFileWriter sst = new SstFileWriter(env, options); @@ -126,17 +120,17 @@ private void createTable(String table) throws RocksDBException { } @Override - public synchronized void dropTable(String... tables) - throws RocksDBException { + public synchronized void dropTable(String... tables) throws RocksDBException { for (String table : tables) { this.dropTable(table); } } - public void dropTable(String table) throws RocksDBException { - SstFileWriter sst = this.tables.remove(table); - assert sst == null || !sst.isOwningHandle() : - "Please close table before drop to ensure call sst.finish()"; + public void dropTable(String table) throws RocksDBException{ + try (SstFileWriter sst = this.tables.remove(table)) { + assert sst == null || !sst.isOwningHandle() : "Please close table before drop to " + + "ensure call sst.finish()"; + } } @Override @@ -155,8 +149,7 @@ public void compactRange() { } @Override - public RocksDBSessions copy(HugeConfig config, - String database, String store) { + public RocksDBSessions copy(HugeConfig config, String database, String store) { return new RocksDBSstSessions(config, database, store, this); } @@ -176,8 +169,7 @@ public String buildSnapshotPath(String snapshotPrefix) { } @Override - public String hardLinkSnapshot(String snapshotPath) - throws RocksDBException { + public String hardLinkSnapshot(String snapshotPath) { throw new UnsupportedOperationException("hardLinkSnapshot"); } @@ -264,7 +256,7 @@ public boolean hasChanges() { @Override public Integer commit() { int count = this.batch.size(); - if (count <= 0) { + if (count == 0) { return 0; } @@ -344,7 +336,7 @@ public void put(String table, byte[] key, byte[] value) { /** * Merge a record to an existing key to a table * For more details about merge-operator: - * https://github.com/facebook/rocksdb/wiki/merge-operator + * ... */ @Override public void merge(String table, byte[] key, byte[] value) { @@ -431,10 +423,8 @@ public BackendColumnIterator scan(String table, byte[] prefix) { * Scan records by key range from a table */ @Override - public BackendColumnIterator scan(String table, - byte[] keyFrom, - byte[] keyTo, - int scanType) { + public BackendColumnIterator scan(String table, byte[] keyFrom, + byte[] keyTo, int scanType) { assert !this.hasChanges(); return BackendColumnIterator.empty(); } diff --git a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstStore.java b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstStore.java index c88cd4970c..12ccfdc15e 100644 --- a/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstStore.java +++ b/hugegraph-server/hugegraph-rocksdb/src/main/java/org/apache/hugegraph/backend/store/rocksdbsst/RocksDBSstStore.java @@ -19,8 +19,6 @@ import java.util.List; -import org.rocksdb.RocksDBException; - import org.apache.hugegraph.backend.id.Id; import org.apache.hugegraph.backend.store.BackendStoreProvider; import org.apache.hugegraph.backend.store.rocksdb.RocksDBSessions; @@ -28,6 +26,7 @@ import org.apache.hugegraph.backend.store.rocksdb.RocksDBTables; import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.type.HugeType; +import org.rocksdb.RocksDBException; public abstract class RocksDBSstStore extends RocksDBStore { @@ -42,8 +41,7 @@ protected RocksDBSessions openSessionPool(HugeConfig config, List tableNames) throws RocksDBException { if (tableNames == null) { - return new RocksDBSstSessions(config, this.database(), - this.store(), dataPath); + return new RocksDBSstSessions(config, this.database(), this.store(), dataPath); } else { return new RocksDBSstSessions(config, this.database(), this.store(), dataPath, tableNames); @@ -58,13 +56,10 @@ public RocksDBSstGraphStore(BackendStoreProvider provider, String database, String store) { super(provider, database, store); - registerTableManager(HugeType.VERTEX, - new RocksDBTables.Vertex(database)); + registerTableManager(HugeType.VERTEX, new RocksDBTables.Vertex(database)); - registerTableManager(HugeType.EDGE_OUT, - RocksDBTables.Edge.out(database)); - registerTableManager(HugeType.EDGE_IN, - RocksDBTables.Edge.in(database)); + registerTableManager(HugeType.EDGE_OUT, RocksDBTables.Edge.out(database)); + registerTableManager(HugeType.EDGE_IN, RocksDBTables.Edge.in(database)); registerTableManager(HugeType.SECONDARY_INDEX, new RocksDBTables.SecondaryIndex(database)); @@ -80,12 +75,9 @@ public RocksDBSstGraphStore(BackendStoreProvider provider, new RocksDBTables.RangeLongIndex(database)); registerTableManager(HugeType.RANGE_DOUBLE_INDEX, new RocksDBTables.RangeDoubleIndex(database)); - registerTableManager(HugeType.SEARCH_INDEX, - new RocksDBTables.SearchIndex(database)); - registerTableManager(HugeType.SHARD_INDEX, - new RocksDBTables.ShardIndex(database)); - registerTableManager(HugeType.UNIQUE_INDEX, - new RocksDBTables.UniqueIndex(database)); + registerTableManager(HugeType.SEARCH_INDEX, new RocksDBTables.SearchIndex(database)); + registerTableManager(HugeType.SHARD_INDEX, new RocksDBTables.ShardIndex(database)); + registerTableManager(HugeType.UNIQUE_INDEX, new RocksDBTables.UniqueIndex(database)); } @Override @@ -95,20 +87,17 @@ public boolean isSchemaStore() { @Override public Id nextId(HugeType type) { - throw new UnsupportedOperationException( - "RocksDBSstGraphStore.nextId()"); + throw new UnsupportedOperationException("RocksDBSstGraphStore.nextId()"); } @Override public void increaseCounter(HugeType type, long increment) { - throw new UnsupportedOperationException( - "RocksDBSstGraphStore.increaseCounter()"); + throw new UnsupportedOperationException("RocksDBSstGraphStore.increaseCounter()"); } @Override public long getCounter(HugeType type) { - throw new UnsupportedOperationException( - "RocksDBSstGraphStore.getCounter()"); + throw new UnsupportedOperationException("RocksDBSstGraphStore.getCounter()"); } } } diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/BaseRocksDBUnitTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/BaseRocksDBUnitTest.java index 9a9104412e..5629938f99 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/BaseRocksDBUnitTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/BaseRocksDBUnitTest.java @@ -23,23 +23,21 @@ import java.util.ArrayList; import org.apache.commons.io.FileUtils; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.rocksdb.RocksDBException; - import org.apache.hugegraph.backend.store.rocksdb.RocksDBSessions; import org.apache.hugegraph.backend.store.rocksdb.RocksDBStdSessions; import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.unit.BaseUnitTest; import org.apache.hugegraph.unit.FakeObjects; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.rocksdb.RocksDBException; public class BaseRocksDBUnitTest extends BaseUnitTest { private static final String TMP_DIR = System.getProperty("java.io.tmpdir"); protected static final String DB_PATH = TMP_DIR + "/" + "rocksdb"; protected static final String SNAPSHOT_PATH = TMP_DIR + "/" + "snapshot"; - protected static final String TABLE = "test-table"; protected RocksDBSessions rocks; @@ -74,10 +72,9 @@ protected String get(String key) throws RocksDBException { return getString(this.rocks.session().get(TABLE, getBytes(key))); } - protected void clearData() throws RocksDBException { + protected void clearData() { for (String table : new ArrayList<>(this.rocks.openedTables())) { - this.rocks.session().deleteRange(table, - new byte[]{0}, new byte[]{-1}); + this.rocks.session().deleteRange(table, new byte[]{0}, new byte[]{-1}); } this.commit(); } @@ -119,7 +116,7 @@ private static RocksDBSessions open(String table) throws RocksDBException { private static void close(RocksDBSessions rocks) throws RocksDBException { for (String table : new ArrayList<>(rocks.openedTables())) { - if (table.equals("default")) { + if ("default".equals(table)) { continue; } rocks.dropTable(table); diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBCountersTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBCountersTest.java index 083cb1d3b4..ee712f9369 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBCountersTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBCountersTest.java @@ -21,16 +21,15 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; -import org.junit.Before; -import org.junit.Test; -import org.rocksdb.RocksDBException; - import org.apache.hugegraph.backend.id.Id; import org.apache.hugegraph.backend.id.IdGenerator; import org.apache.hugegraph.backend.store.rocksdb.RocksDBSessions.Session; import org.apache.hugegraph.backend.store.rocksdb.RocksDBTables; import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.type.HugeType; +import org.junit.Before; +import org.junit.Test; +import org.rocksdb.RocksDBException; public class RocksDBCountersTest extends BaseRocksDBUnitTest { @@ -104,7 +103,7 @@ public void testCounterWithMutiThreads() { private Id nextId(Session session, HugeType type) { final int MAX_TIMES = 1000; // Do get-increase-get-compare operation - long counter = 0L; + long counter; long expect = -1L; synchronized (this) { for (int i = 0; i < MAX_TIMES; i++) { diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBPerfTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBPerfTest.java index a6d94d1b8a..fdab91e916 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBPerfTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBPerfTest.java @@ -24,19 +24,17 @@ import java.util.Map; import java.util.Random; -import org.junit.Test; -import org.rocksdb.RocksDBException; - import org.apache.hugegraph.backend.store.BackendEntry.BackendColumn; import org.apache.hugegraph.backend.store.BackendEntry.BackendColumnIterator; import org.apache.hugegraph.backend.store.rocksdb.RocksDBSessions.Session; +import org.junit.Test; public class RocksDBPerfTest extends BaseRocksDBUnitTest { private static final int TIMES = 10000 * 1000; @Test - public void testSeekExistKey() throws RocksDBException { + public void testSeekExistKey() { put("exist", "value"); Session session = this.rocks.session(); @@ -49,7 +47,7 @@ public void testSeekExistKey() throws RocksDBException { } @Test - public void testSeekNonExistKey() throws RocksDBException { + public void testSeekNonExistKey() { put("exist", "value"); Session session = this.rocks.session(); @@ -62,7 +60,7 @@ public void testSeekNonExistKey() throws RocksDBException { } @Test - public void testGetExistKey() throws RocksDBException { + public void testGetExistKey() { put("exist", "value"); Session session = this.rocks.session(); @@ -73,7 +71,7 @@ public void testGetExistKey() throws RocksDBException { } @Test - public void testGetNonExistKey() throws RocksDBException { + public void testGetNonExistKey() { put("exist", "value"); Session session = this.rocks.session(); @@ -84,14 +82,14 @@ public void testGetNonExistKey() throws RocksDBException { } @Test - public void testPut() throws RocksDBException { + public void testPut() { for (int i = 0; i < TIMES; i++) { put("person-" + i, "value-" + i); } } @Test - public void testGet3Keys() throws RocksDBException { + public void testGet3Keys() { put("person:1gname", "James"); put("person:1gage", "19"); @@ -110,7 +108,7 @@ public void testGet3Keys() throws RocksDBException { } @Test - public void testMultiGet3Keys() throws RocksDBException { + public void testMultiGet3Keys() { put("person:1gname", "James"); put("person:1gage", "19"); @@ -134,7 +132,7 @@ public void testMultiGet3Keys() throws RocksDBException { } @Test - public void testGet1KeyWithMultiValues() throws RocksDBException { + public void testGet1KeyWithMultiValues() { put("person:1gname", "James"); put("person:1gage", "19"); @@ -153,7 +151,7 @@ public void testGet1KeyWithMultiValues() throws RocksDBException { } @Test - public void testScanByPrefix() throws RocksDBException { + public void testScanByPrefix() { put("person:1gname", "James"); put("person:1gage", "19"); @@ -173,31 +171,31 @@ public void testScanByPrefix() throws RocksDBException { } @Test - public void testGet3KeysWithData() throws RocksDBException { + public void testGet3KeysWithData() { testPut(); testGet3Keys(); } @Test - public void testMultiGet3KeysWithData() throws RocksDBException { + public void testMultiGet3KeysWithData() { testPut(); testMultiGet3Keys(); } @Test - public void testGet1KeyWithData() throws RocksDBException { + public void testGet1KeyWithData() { testPut(); testGet1KeyWithMultiValues(); } @Test - public void testScanByPrefixWithData() throws RocksDBException { + public void testScanByPrefixWithData() { testPut(); testScanByPrefix(); } @Test - public void testUpdate() throws RocksDBException { + public void testUpdate() { Session session = this.rocks.session(); Random r = new Random(); @@ -231,7 +229,7 @@ public void testUpdate() throws RocksDBException { } @Test - public void testScanByPrefixAfterUpdate() throws RocksDBException { + public void testScanByPrefixAfterUpdate() { Session session = this.rocks.session(); this.testUpdate(); diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionTest.java index 839a0b3e0f..94ffe22949 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionTest.java @@ -19,27 +19,27 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.Random; -import org.apache.hugegraph.unit.BaseUnitTest; -import org.junit.Assume; -import org.junit.Test; -import org.rocksdb.RocksDBException; - import org.apache.hugegraph.backend.store.BackendEntry.BackendColumn; import org.apache.hugegraph.backend.store.BackendEntry.BackendColumnIterator; import org.apache.hugegraph.backend.store.rocksdb.RocksDBSessions.Session; import org.apache.hugegraph.testutil.Assert; +import org.apache.hugegraph.unit.BaseUnitTest; +import org.junit.Assume; +import org.junit.Test; +import org.rocksdb.RocksDBException; public class RocksDBSessionTest extends BaseRocksDBUnitTest { @Test - public void testPutAndGet() throws RocksDBException { + public void testPutAndGet() { String value = getString(this.rocks.session().get(TABLE, getBytes("person:1gname"))); - Assert.assertEquals(null, value); + Assert.assertNull(value); this.rocks.session().put(TABLE, getBytes("person:1gname"), getBytes("James")); this.rocks.session().put(TABLE, getBytes("person:1gage"), getBytes(19)); @@ -57,9 +57,9 @@ public void testPutAndGet() throws RocksDBException { } @Test - public void testPutAndMultiGet() throws RocksDBException { - BackendColumnIterator values = this.rocks.session().get(TABLE, - Arrays.asList(getBytes("person:1gname"))); + public void testPutAndMultiGet() { + BackendColumnIterator values = + this.rocks.session().get(TABLE, Collections.singletonList(getBytes("person:1gname"))); Assert.assertFalse(values.hasNext()); this.rocks.session().put(TABLE, getBytes("person:1gname"), getBytes("James")); @@ -67,9 +67,8 @@ public void testPutAndMultiGet() throws RocksDBException { this.rocks.session().put(TABLE, getBytes("person:1gcity"), getBytes("Beijing")); this.commit(); - values = this.rocks.session().get(TABLE, Arrays.asList( - getBytes("person:1gname"), - getBytes("person:1gage"))); + values = this.rocks.session().get(TABLE, Arrays.asList(getBytes("person:1gname"), + getBytes("person:1gage"))); Assert.assertTrue(values.hasNext()); Assert.assertEquals("James", getString(values.next().value)); Assert.assertEquals(19, getLong(values.next().value)); @@ -123,7 +122,7 @@ public void testPutAndGetWithMultiTables() throws RocksDBException { } @Test - public void testMergeWithCounter() throws RocksDBException { + public void testMergeWithCounter() { this.rocks.session().put(TABLE, getBytes("person:1gage"), getBytes(19)); this.commit(); @@ -163,7 +162,7 @@ public void testMergeWithStringList() throws RocksDBException { } @Test - public void testScanByAll() throws RocksDBException { + public void testScanByAll() { put("person:1gname", "James"); put("person:2gname", "Lisa"); @@ -397,7 +396,7 @@ public void testDeleteByKey() throws RocksDBException { this.commit(); Assert.assertEquals("James", get("person:1gname")); - Assert.assertEquals(null, get("person:1gage")); + Assert.assertNull(get("person:1gage")); Assert.assertEquals("Beijing", get("person:1gcity")); } @@ -436,9 +435,9 @@ public void testDeleteByPrefix() throws RocksDBException { this.rocks.session().deletePrefix(TABLE, getBytes("person:1")); this.commit(); - Assert.assertEquals(null, get("person:1gname")); - Assert.assertEquals(null, get("person:1gage")); - Assert.assertEquals(null, get("person:1gcity")); + Assert.assertNull(get("person:1gname")); + Assert.assertNull(get("person:1gage")); + Assert.assertNull(get("person:1gcity")); Assert.assertEquals("Lisa", get("person:2gname")); } @@ -464,13 +463,13 @@ public void testDeleteByRange() throws RocksDBException { this.rocks.session().deleteRange(TABLE, getBytes("person:1"), getBytes("person:3")); this.commit(); - Assert.assertEquals(null, get("person:1gname")); - Assert.assertEquals(null, get("person:1gage")); - Assert.assertEquals(null, get("person:1gcity")); + Assert.assertNull(get("person:1gname")); + Assert.assertNull(get("person:1gage")); + Assert.assertNull(get("person:1gcity")); - Assert.assertEquals(null, get("person:2gname")); - Assert.assertEquals(null, get("person:2gage")); - Assert.assertEquals(null, get("person:2gcity")); + Assert.assertNull(get("person:2gname")); + Assert.assertNull(get("person:2gage")); + Assert.assertNull(get("person:2gcity")); Assert.assertEquals("Hebe", get("person:3gname")); Assert.assertEquals("21", get("person:3gage")); @@ -543,7 +542,7 @@ public void testDeleteByRangeWithSignedBytes() throws RocksDBException { } @Test - public void testDeleteByRangeWithMinMaxByteValue() throws RocksDBException { + public void testDeleteByRangeWithMinMaxByteValue() { Session session = this.rocks.session(); byte[] key11 = new byte[]{1, 0}; @@ -601,17 +600,17 @@ public void testDeleteSingle() throws RocksDBException { this.commit(); Assert.assertEquals("James2", get("person:1gname")); - Assert.assertEquals(null, get("person:1gage")); + Assert.assertNull(get("person:1gage")); // deleteSingle after put twice this.rocks.session().deleteSingle(TABLE, getBytes("person:1gname")); this.commit(); // NOTE: maybe return "James" here - Assert.assertEquals(null, get("person:1gname")); + Assert.assertNull(get("person:1gname")); Assert.assertTrue(null == get("person:1gname") || "James".equals(get("person:1gname"))); - Assert.assertEquals(null, get("person:1gage")); + Assert.assertNull(get("person:1gage")); } @Test @@ -628,13 +627,13 @@ public void testCompact() throws RocksDBException { this.commit(); Assert.assertEquals("James", get("person:1gname")); - Assert.assertEquals(null, get("person:1gage")); + Assert.assertNull(get("person:1gage")); Assert.assertEquals("Beijing", get("person:1gcity")); this.rocks.session().compactRange(TABLE); Assert.assertEquals("James", get("person:1gname")); - Assert.assertEquals(null, get("person:1gage")); + Assert.assertNull(get("person:1gage")); Assert.assertEquals("Beijing", get("person:1gcity")); } diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionsTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionsTest.java index 37e1472c42..aa74d9cd64 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionsTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/rocksdb/RocksDBSessionsTest.java @@ -21,9 +21,6 @@ import java.io.IOException; import org.apache.commons.io.FileUtils; -import org.junit.Test; -import org.rocksdb.RocksDBException; - import org.apache.hugegraph.backend.store.rocksdb.RocksDBMetrics; import org.apache.hugegraph.backend.store.rocksdb.RocksDBOptions; import org.apache.hugegraph.backend.store.rocksdb.RocksDBSessions; @@ -32,6 +29,9 @@ import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.unit.FakeObjects; +import org.junit.Test; +import org.rocksdb.RocksDBException; + import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -162,9 +162,7 @@ public void testIngestSst() throws RocksDBException { HugeConfig config = FakeObjects.newConfig(); String sstPath = DB_PATH + "/sst"; config.addProperty(RocksDBOptions.SST_PATH.name(), sstPath); - RocksDBSstSessions sstSessions = new RocksDBSstSessions(config, - "sst", "store", - sstPath); + RocksDBSstSessions sstSessions = new RocksDBSstSessions(config, "sst", "store", sstPath); final String TABLE1 = "test-table1"; final String TABLE2 = "test-table2"; sstSessions.createTable(TABLE1); @@ -192,8 +190,7 @@ public void testIngestSst() throws RocksDBException { Assert.assertFalse(sstSessions.existsTable(TABLE1)); Assert.assertFalse(sstSessions.existsTable(TABLE2)); - RocksDBSessions rocks = new RocksDBStdSessions(config, "db", "store", - sstPath, sstPath); + RocksDBSessions rocks = new RocksDBStdSessions(config, "db", "store", sstPath, sstPath); // Will ingest sst file of TABLE1 rocks.createTable(TABLE1); Assert.assertEquals(ImmutableList.of("1000"), diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryBackendEntryTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryBackendEntryTest.java index 2ead8ba58a..97b55e30b7 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryBackendEntryTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryBackendEntryTest.java @@ -17,13 +17,13 @@ package org.apache.hugegraph.unit.serializer; -import org.junit.Test; - import org.apache.hugegraph.backend.serializer.BinaryBackendEntry; import org.apache.hugegraph.backend.store.BackendEntry.BackendColumn; import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.type.HugeType; import org.apache.hugegraph.unit.BaseUnitTest; +import org.junit.Test; + import com.google.common.collect.ImmutableList; public class BinaryBackendEntryTest extends BaseUnitTest { diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryScatterSerializerTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryScatterSerializerTest.java index 28a6a219df..abc1a92fa2 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryScatterSerializerTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinaryScatterSerializerTest.java @@ -17,18 +17,17 @@ package org.apache.hugegraph.unit.serializer; -import org.apache.hugegraph.config.HugeConfig; -import org.junit.Test; - import org.apache.hugegraph.backend.serializer.BinaryBackendEntry; import org.apache.hugegraph.backend.serializer.BinaryScatterSerializer; import org.apache.hugegraph.backend.store.BackendEntry; +import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.structure.HugeEdge; import org.apache.hugegraph.structure.HugeVertex; import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.testutil.Whitebox; import org.apache.hugegraph.unit.BaseUnitTest; import org.apache.hugegraph.unit.FakeObjects; +import org.junit.Test; public class BinaryScatterSerializerTest extends BaseUnitTest { @@ -84,8 +83,7 @@ public void testEdge() { private static BackendEntry parse(BackendEntry originEntry) { byte[] bytes = originEntry.id().asBytes(); - BackendEntry parsedEntry = new BinaryBackendEntry(originEntry.type(), - bytes); + BackendEntry parsedEntry = new BinaryBackendEntry(originEntry.type(), bytes); parsedEntry.columns(originEntry.columns()); return parsedEntry; } diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinarySerializerTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinarySerializerTest.java index 3eb269a265..59e77eb5dd 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinarySerializerTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/serializer/BinarySerializerTest.java @@ -17,17 +17,16 @@ package org.apache.hugegraph.unit.serializer; -import org.apache.hugegraph.config.HugeConfig; -import org.junit.Test; - import org.apache.hugegraph.backend.serializer.BinarySerializer; import org.apache.hugegraph.backend.store.BackendEntry; +import org.apache.hugegraph.config.HugeConfig; import org.apache.hugegraph.structure.HugeEdge; import org.apache.hugegraph.structure.HugeVertex; import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.testutil.Whitebox; import org.apache.hugegraph.unit.BaseUnitTest; import org.apache.hugegraph.unit.FakeObjects; +import org.junit.Test; public class BinarySerializerTest extends BaseUnitTest { From b52517cc47a1a24ee08bf758442069bd1f8c30d3 Mon Sep 17 00:00:00 2001 From: conghuhu <56248584+conghuhu@users.noreply.github.com> Date: Mon, 11 Dec 2023 22:25:51 -0600 Subject: [PATCH 7/9] feat(core): add IntMapByDynamicHash V1 implement (#2377) * feat(WIP): add IntMapByDynamicHash (#2294) * feat: add values & keys in IntMapByDynamicHash * add some basic comment & fix some style * feat: fix pr review * fix: fix some review --------- Co-authored-by: imbajin --- .../util/collection/IntMapByDynamicHash.java | 1022 +++++++++++++++++ .../unit/util/collection/IntMapTest.java | 115 +- .../benchmark/BenchmarkConstants.java | 2 +- .../map/MapRandomGetPutThroughputTest.java | 60 +- 4 files changed, 1170 insertions(+), 29 deletions(-) create mode 100644 hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/util/collection/IntMapByDynamicHash.java diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/util/collection/IntMapByDynamicHash.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/util/collection/IntMapByDynamicHash.java new file mode 100644 index 0000000000..a86def7f13 --- /dev/null +++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/util/collection/IntMapByDynamicHash.java @@ -0,0 +1,1022 @@ +/* + * 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 org.apache.hugegraph.util.collection; + +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; + +import sun.misc.Unsafe; + +/** + * This class implements a concurrent hash map specifically designed for integer keys and values. + * It uses low-level programming techniques such as direct memory access via `sun.misc.Unsafe` to + * achieve high performance. + * The class is part of the Apache HugeGraph project. + */ +public class IntMapByDynamicHash implements IntMap { + + private static final int DEFAULT_INITIAL_CAPACITY = 16; + + /** + * The maximum capacity, used if a higher value is implicitly specified + * by either of the constructors with arguments. + * MUST be a power of two <= 1<<30. + */ + private static final int MAXIMUM_CAPACITY = 1 << 30; + + private static final float LOAD_FACTOR = 0.75f; + + private static final int PARTITIONED_SIZE_THRESHOLD = 4096; + + private static final int NULL_VALUE = Integer.MIN_VALUE; + + private static final AtomicReferenceFieldUpdater + TABLE_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(IntMapByDynamicHash.class, Entry[].class, "table"); + + private volatile Entry[] table; + + /** + * Partition counting to improve the concurrency performance of addToSize() + */ + private int[] partitionedSize; + + /** + * updated via atomic field updater + */ + @SuppressWarnings("UnusedDeclaration") + private volatile int size; + + private static final Entry RESIZING = new Entry(NULL_VALUE, NULL_VALUE, (byte) 1); + private static final Entry RESIZED = new Entry(NULL_VALUE, NULL_VALUE, (byte) 2); + + private static final Entry RESIZE_SENTINEL = new Entry(NULL_VALUE, NULL_VALUE, (byte) 3); + + /** + * must be (2^n) - 1 + */ + private static final int SIZE_BUCKETS = 7; + + /** + * Constructor for the IntMapByDynamicHash class. + * + * @param initialCapacity the initial capacity of the map. + */ + public IntMapByDynamicHash(int initialCapacity) { + if (initialCapacity < 0) { + throw new IllegalArgumentException("Illegal Initial Capacity: " + initialCapacity); + } + if (initialCapacity > MAXIMUM_CAPACITY) { + initialCapacity = MAXIMUM_CAPACITY; + } + long size = (long) (1.0 + (long) initialCapacity / LOAD_FACTOR); + int cap = (size >= (long) MAXIMUM_CAPACITY) ? + MAXIMUM_CAPACITY : tableSizeFor((int) size); + if (cap >= PARTITIONED_SIZE_THRESHOLD) { + // we want 7 extra slots, and 64 bytes for each slot int are 4 bytes, + // so 64 bytes are 16 ints. + this.partitionedSize = + new int[SIZE_BUCKETS * 16]; + } + // The end index is for resizeContainer + this.table = new Entry[cap + 1]; + } + + /** + * Default constructor for the IntMapByDynamicHash class. + * Initializes the map with the default initial capacity. + */ + public IntMapByDynamicHash() { + this(DEFAULT_INITIAL_CAPACITY); + } + + private static void setTableAt(Object[] array, int index, Object newValue) { + UNSAFE.putObjectVolatile(array, ((long) index << ENTRY_ARRAY_SHIFT) + ENTRY_ARRAY_BASE, + newValue); + } + + private static int tableSizeFor(int c) { + int n = c - 1; + n |= n >>> 1; + n |= n >>> 2; + n |= n >>> 4; + n |= n >>> 8; + n |= n >>> 16; + return (n < 0) ? 1 : (n >= MAXIMUM_CAPACITY) ? MAXIMUM_CAPACITY : n + 1; + } + + /* ---------------- Table element access -------------- */ + + private static long entryOffset(int index) { + return ((long) index << ENTRY_ARRAY_SHIFT) + ENTRY_ARRAY_BASE; + } + + private static Object tableAt(Object[] array, int index) { + return UNSAFE.getObjectVolatile(array, entryOffset(index)); + } + + private static boolean casTableAt(Object[] array, int index, Object expected, Object newValue) { + return UNSAFE.compareAndSwapObject(array, entryOffset(index), expected, newValue); + } + + /** + * Puts a key-value pair into the map. If the key already exists in the map, its value is + * updated. + * + * @param key the key to be put into the map. + * @param value the value to be associated with the key. + * @return true if the operation is successful. + */ + @Override + public boolean put(int key, int value) { + int hash = this.hash(key); + Entry[] currentArray = this.table; + Entry o = (Entry) IntMapByDynamicHash.tableAt(currentArray, hash); + if (o == null) { + Entry newEntry = new Entry(key, value); + this.addToSize(1); + if (IntMapByDynamicHash.casTableAt(currentArray, hash, null, newEntry)) { + return true; + } + this.addToSize(-1); + } + + this.slowPut(key, value, currentArray); + return true; + } + + /** + * This method is used when the normal put operation fails due to a hash collision. + * It searches for the key in the chain and if found, replaces the entry. + * If the key is not found, it adds a new entry. + * + * @param key the key to be put into the map. + * @param value the value to be associated with the key. + * @param currentTable the current table where the key-value pair is to be put. + * @return the old value if the key is already present in the map, otherwise NULL_VALUE. + */ + private int slowPut(int key, int value, Entry[] currentTable) { + int length; + int index; + Entry o; + + while (true) { + length = currentTable.length; + index = this.hash(key, length); + o = (Entry) IntMapByDynamicHash.tableAt(currentTable, index); + + if (o == RESIZED || o == RESIZING) { + currentTable = this.helpWithResizeWhileCurrentIndex(currentTable, index); + } else { + Entry e = o; + boolean found = false; + + // Search for the key in the chain + while (e != null) { + int candidate = e.getKey(); + if (candidate == key) { + found = true; + break; + } + e = e.getNext(); + } + + if (found) { + int oldVal = e.getValue(); + // Key found, replace the entry + Entry newEntry = + new Entry(key, value, this.createReplacementChainForRemoval(o, e)); + if (IntMapByDynamicHash.casTableAt(currentTable, index, o, newEntry)) { + return oldVal; + } + } else { + // Key not found, add a new entry + Entry newEntry = new Entry(key, value, o); + if (IntMapByDynamicHash.casTableAt(currentTable, index, o, newEntry)) { + this.incrementSizeAndPossiblyResize(currentTable, length, o); + return NULL_VALUE; + } + } + } + } + } + + /** + * Retrieves the value associated with the given key from the map. + * + * @param key the key whose associated value is to be returned. + * @return the value associated with the given key, or NULL_VALUE if the key does not exist + * in the map. + */ + @Override + public int get(int key) { + int hash = this.hash(key); + Entry[] currentArray = this.table; + Entry o = (Entry) IntMapByDynamicHash.tableAt(currentArray, hash); + if (o == RESIZED || o == RESIZING) { + return this.slowGet(key, currentArray); + } + for (Entry e = o; e != null; e = e.getNext()) { + int k; + // TODO: check why key == k is always false + if ((k = e.getKey()) == key || key == k) { + return e.value; + } + } + return NULL_VALUE; + } + + /** + * This method is used when the normal get operation fails due to a hash collision. + * It searches for the key in the chain and returns the associated value if found. + * + * @param key the key whose associated value is to be returned. + * @param currentArray the current table where the key-value pair is located. + * @return the value associated with the given key, or NULL_VALUE if the key does not exist + * in the map. + */ + private int slowGet(int key, Entry[] currentArray) { + while (true) { + int length = currentArray.length; + int hash = this.hash(key, length); + Entry o = (Entry) IntMapByDynamicHash.tableAt(currentArray, hash); + if (o == RESIZED || o == RESIZING) { + currentArray = this.helpWithResizeWhileCurrentIndex(currentArray, hash); + } else { + Entry e = o; + while (e != null) { + int candidate = e.getKey(); + if (candidate == key) { + return e.getValue(); + } + e = e.getNext(); + } + return NULL_VALUE; + } + } + } + + /** + * Removes the key-value pair with the given key from the map. + * + * @param key the key whose associated key-value pair is to be removed. + * @return true if the key-value pair was found and removed, false otherwise. + */ + @Override + public boolean remove(int key) { + int hash = this.hash(key); + Entry[] currentTable = this.table; + Entry o = (Entry) IntMapByDynamicHash.tableAt(currentTable, hash); + if (o == RESIZED || o == RESIZING) { + return this.slowRemove(key, currentTable) != null; + } + + Entry e = o; + while (e != null) { + int candidate = e.getKey(); + if (candidate == key) { + Entry replacement = this.createReplacementChainForRemoval(o, e); + if (IntMapByDynamicHash.casTableAt(currentTable, hash, o, replacement)) { + this.addToSize(-1); + return true; + } + return this.slowRemove(key, currentTable) != null; + } + e = e.getNext(); + } + return false; + } + + /** + * This method is used when the normal remove operation fails due to a hash collision. + * It searches for the key in the chain and if found, removes the entry. + * + * @param key the key whose associated key-value pair is to be removed. + * @param currentTable the current table where the key-value pair is located. + * @return the removed entry if the key is found, otherwise null. + */ + private Entry slowRemove(int key, Entry[] currentTable) { + int length; + int index; + Entry o; + + while (true) { + length = currentTable.length; + index = this.hash(key, length); + o = (Entry) IntMapByDynamicHash.tableAt(currentTable, index); + if (o == RESIZED || o == RESIZING) { + currentTable = this.helpWithResizeWhileCurrentIndex(currentTable, index); + } else { + Entry e = o; + Entry prev = null; + + while (e != null) { + int candidate = e.getKey(); + if (candidate == key) { + Entry replacement = this.createReplacementChainForRemoval(o, e); + if (IntMapByDynamicHash.casTableAt(currentTable, index, o, replacement)) { + this.addToSize(-1); + return e; + } + // Key found, but CAS failed, restart the loop + break; + } + prev = e; + e = e.getNext(); + } + + if (prev != null) { + // Key doesn't found + return null; + } + } + } + } + + /** + * Checks if the map contains a key-value pair with the given key. + * + * @param key the key to be checked. + * @return true if the map contains a key-value pair with the given key, false otherwise. + */ + @Override + public boolean containsKey(int key) { + return this.getEntry(key) != null; + } + + @Override + public IntIterator keys() { + return new KeyIterator(); + } + + @Override + public IntIterator values() { + return new ValueIterator(); + } + + /** + * Removes all the mappings from this map. The map will be empty after this call returns. + */ + @Override + public void clear() { + Entry[] currentArray = this.table; + ResizeContainer resizeContainer; + do { + resizeContainer = null; + for (int i = 0; i < currentArray.length - 1; i++) { + Entry o = (Entry) IntMapByDynamicHash.tableAt(currentArray, i); + if (o == RESIZED || o == RESIZING) { + resizeContainer = + (ResizeContainer) IntMapByDynamicHash.tableAt(currentArray, + currentArray.length - 1); + } else if (o != null) { + Entry e = o; + if (IntMapByDynamicHash.casTableAt(currentArray, i, o, null)) { + int removedEntries = 0; + while (e != null) { + removedEntries++; + e = e.getNext(); + } + this.addToSize(-removedEntries); + } + } + } + if (resizeContainer != null) { + if (resizeContainer.isNotDone()) { + this.helpWithResize(currentArray); + resizeContainer.waitForAllResizers(); + } + currentArray = resizeContainer.nextArray; + } + } while (resizeContainer != null); + } + + @Override + public int size() { + int localSize = this.size; + if (this.partitionedSize != null) { + for (int i = 0; i < SIZE_BUCKETS; i++) { + localSize += this.partitionedSize[i << 4]; + } + } + return localSize; + } + + @Override + public boolean concurrent() { + return true; + } + + private int hash(int key) { + return key & (table.length - 2); + } + + private int hash(int key, int length) { + return key & (length - 2); + } + + private Entry getEntry(int key) { + Entry[] currentArray = this.table; + while (true) { + int length = currentArray.length; + int index = this.hash(key, length); + Entry o = (Entry) IntMapByDynamicHash.tableAt(currentArray, index); + if (o == RESIZED || o == RESIZING) { + currentArray = this.helpWithResizeWhileCurrentIndex(currentArray, index); + } else { + Entry e = o; + while (e != null) { + int candidate = e.getKey(); + if (candidate == key) { + return e; + } + e = e.getNext(); + } + return null; + } + } + } + + private void addToSize(int value) { + if (this.partitionedSize != null) { + if (this.incrementPartitionedSize(value)) { + return; + } + } + this.incrementLocalSize(value); + } + + private boolean incrementPartitionedSize(int value) { + int h = (int) Thread.currentThread().getId(); + h ^= (h >>> 18) ^ (h >>> 12); + h = (h ^ (h >>> 10)) & SIZE_BUCKETS; + if (h != 0) { + h = (h - 1) << 4; + long address = ((long) h << INT_ARRAY_SHIFT) + INT_ARRAY_BASE; + while (true) { + int localSize = UNSAFE.getIntVolatile(this.partitionedSize, address); + if (UNSAFE.compareAndSwapInt(this.partitionedSize, address, localSize, + localSize + value)) { + return true; + } + } + } + return false; + } + + private void incrementLocalSize(int value) { + while (true) { + int localSize = this.size; + if (UNSAFE.compareAndSwapInt(this, SIZE_OFFSET, localSize, localSize + value)) { + break; + } + } + } + + private Entry createReplacementChainForRemoval(Entry original, Entry toRemove) { + if (original == toRemove) { + return original.getNext(); + } + Entry replacement = null; + Entry e = original; + while (e != null) { + if (e != toRemove) { + replacement = new Entry(e.getKey(), e.getValue(), replacement); + } + e = e.getNext(); + } + return replacement; + } + + private void incrementSizeAndPossiblyResize(Entry[] currentArray, int length, Entry prev) { + this.addToSize(1); + if (prev != null) { + int localSize = this.size(); + int threshold = (int) (length * LOAD_FACTOR); // threshold = length * 0.75 + if (localSize + 1 > threshold) { + this.resize(currentArray); + } + } + } + + private Entry[] helpWithResizeWhileCurrentIndex(Entry[] currentArray, int index) { + Entry[] newArray = this.helpWithResize(currentArray); + int helpCount = 0; + while (IntMapByDynamicHash.tableAt(currentArray, index) != RESIZED) { + helpCount++; + newArray = this.helpWithResize(currentArray); + if ((helpCount & 7) == 0) { + Thread.yield(); + } + } + return newArray; + } + + private void resize(Entry[] oldTable) { + this.resize(oldTable, (oldTable.length - 1 << 1) + 1); + } + + /** + * Resizes the map to a new capacity. This method is called when the map's size exceeds its + * threshold. It creates a new array with the new capacity and transfers all entries from the + * old array to the new one. + * Note: newSize must be a power of 2 + 1 + * + * @param oldTable The old table to resize. + * @param newSize The new size for the table. + */ + @SuppressWarnings("JLM_JSR166_UTILCONCURRENT_MONITORENTER") + private void resize(Entry[] oldTable, int newSize) { + int oldCapacity = oldTable.length; + int end = oldCapacity - 1; + Entry last = (Entry) IntMapByDynamicHash.tableAt(oldTable, end); + if (this.size() < end && last == RESIZE_SENTINEL) { + return; + } + if (oldCapacity >= MAXIMUM_CAPACITY) { + throw new RuntimeException("max capacity of map exceeded"); + } + ResizeContainer resizeContainer = null; + // This ownResize records whether current thread need to perform the expansion operation of + // the map by itself + boolean ownResize = false; + if (last == null || last == RESIZE_SENTINEL) { + // allocating a new array is too expensive to make this an atomic operation + synchronized (oldTable) { + if (IntMapByDynamicHash.tableAt(oldTable, end) == null) { + IntMapByDynamicHash.setTableAt(oldTable, end, RESIZE_SENTINEL); + if (this.partitionedSize == null && newSize >= PARTITIONED_SIZE_THRESHOLD) { + this.partitionedSize = new int[SIZE_BUCKETS * 16]; + } + resizeContainer = new ResizeContainer(new Entry[newSize], oldTable.length - 1); + IntMapByDynamicHash.setTableAt(oldTable, end, resizeContainer); + ownResize = true; + } + } + } + if (ownResize) { + this.transfer(oldTable, resizeContainer); + + Entry[] src = this.table; + while (!TABLE_UPDATER.compareAndSet(this, oldTable, resizeContainer.nextArray)) { + /* + we're in a double resize situation; we'll have to go help until it's our turn + to set the table + */ + if (src != oldTable) { + this.helpWithResize(src); + } + } + } else { + this.helpWithResize(oldTable); + } + } + + /** + * Transfers all entries from the source table to the destination table. This method is + * called during the resize operation. It iterates over the source table and for each non-null + * entry, it copies the entry to the destination table. If the entry in the source table is + * marked as RESIZED or RESIZING, it helps with the resize operation. + * After all entries are transferred, it notifies the ResizeContainer that the resize operation + * is done. + * + * @param src The source table from which entries are to be transferred. + * @param resizeContainer The container that holds the state of the resize operation. + */ + private void transfer(Entry[] src, ResizeContainer resizeContainer) { + Entry[] dest = resizeContainer.nextArray; + + for (int j = 0; j < src.length - 1; ) { + Entry o = (Entry) IntMapByDynamicHash.tableAt(src, j); + if (o == null) { + if (IntMapByDynamicHash.casTableAt(src, j, null, RESIZED)) { + j++; + } + } else if (o == RESIZED || o == RESIZING) { + /* + During the expansion process, other threads have already migrated the elements at + this location to the new array. This means that the elements in the current + position have already been processed and do not need to be migrated again. + */ + j = (j & ~(ResizeContainer.QUEUE_INCREMENT - 1)) + ResizeContainer.QUEUE_INCREMENT; + /* + When there is only one thread for expansion, there is no concurrency issue + and there is no need to wait. + */ + if (resizeContainer.resizers.get() == 1) { + break; + } + } else { + Entry e = o; + if (IntMapByDynamicHash.casTableAt(src, j, o, RESIZING)) { + while (e != null) { + this.unconditionalCopy(dest, e); + e = e.getNext(); + } + IntMapByDynamicHash.setTableAt(src, j, RESIZED); + j++; + } + } + } + resizeContainer.decrementResizerAndNotify(); + resizeContainer.waitForAllResizers(); + } + + /** + * Enable the current thread to participate in the expansion + */ + private Entry[] helpWithResize(Entry[] currentArray) { + ResizeContainer resizeContainer = + (ResizeContainer) IntMapByDynamicHash.tableAt(currentArray, currentArray.length - 1); + Entry[] newTable = resizeContainer.nextArray; + if (resizeContainer.getQueuePosition() > ResizeContainer.QUEUE_INCREMENT) { + resizeContainer.incrementResizer(); + this.reverseTransfer(currentArray, resizeContainer); + resizeContainer.decrementResizerAndNotify(); + } + return newTable; + } + + /** + * Transfers entries from the old table to the new table in reverse order. This method is used + * to help the resize operation by spreading the work among multiple threads. Each thread + * transfers a portion of the entries from the end of the old table to the beginning of the new + * table. + * + * @param src The old table to transfer entries from. + * @param resizeContainer The container that holds the state of the resize operation. + */ + private void reverseTransfer(Entry[] src, ResizeContainer resizeContainer) { + Entry[] dest = resizeContainer.nextArray; + while (resizeContainer.getQueuePosition() > 0) { + int start = resizeContainer.subtractAndGetQueuePosition(); + int end = start + ResizeContainer.QUEUE_INCREMENT; + if (end > 0) { + if (start < 0) { + start = 0; + } + for (int j = end - 1; j >= start; ) { + Entry o = (Entry) IntMapByDynamicHash.tableAt(src, j); + if (o == null) { + if (IntMapByDynamicHash.casTableAt(src, j, null, RESIZED)) { + j--; + } + } else if (o == RESIZED || o == RESIZING) { + resizeContainer.zeroOutQueuePosition(); + return; + } else { + Entry e = o; + if (IntMapByDynamicHash.casTableAt(src, j, o, RESIZING)) { + while (e != null) { + this.unconditionalCopy(dest, e); + e = e.getNext(); + } + IntMapByDynamicHash.setTableAt(src, j, RESIZED); + j--; + } + } + } + } + } + } + + /** + * Copies an entry from the old table to the new table. This method is called during the resize + * operation. It does not check if the entry already exists in the new table, so it should only + * be called with entries that are not in the new table yet. + * + * @param dest The new table to copy the entry to. + * @param toCopyEntry The entry to copy. + */ + private void unconditionalCopy(Entry[] dest, Entry toCopyEntry) { + Entry[] currentArray = dest; + while (true) { + int length = currentArray.length; + int index = this.hash(toCopyEntry.getKey(), length); + Entry o = (Entry) IntMapByDynamicHash.tableAt(currentArray, index); + if (o == RESIZED || o == RESIZING) { + currentArray = + ((ResizeContainer) IntMapByDynamicHash.tableAt(currentArray, + length - 1)).nextArray; + } else { + Entry newEntry; + if (o == null) { + if (toCopyEntry.getNext() == null) { + newEntry = toCopyEntry; // no need to duplicate + } else { + newEntry = new Entry(toCopyEntry.getKey(), toCopyEntry.getValue()); + } + } else { + newEntry = new Entry(toCopyEntry.getKey(), toCopyEntry.getValue(), o); + } + if (IntMapByDynamicHash.casTableAt(currentArray, index, o, newEntry)) { + return; + } + } + } + } + + /** + * The ResizeContainer class is used to hold the state of the resize operation. + * It contains the new array to which entries are transferred, the number of threads + * participating in the resize operation, and the position in the old array from which + * entries are transferred. + */ + private static final class ResizeContainer extends Entry { + + private static final int QUEUE_INCREMENT = + Math.min(1 << 10, + Integer.highestOneBit(IntSet.CPUS) << 4); + private final AtomicInteger resizers = new AtomicInteger(1); + private final Entry[] nextArray; + private final AtomicInteger queuePosition; + + private ResizeContainer(Entry[] nextArray, int oldSize) { + super(NULL_VALUE, NULL_VALUE, (byte) 4); + this.nextArray = nextArray; + this.queuePosition = new AtomicInteger(oldSize); + } + + public void incrementResizer() { + this.resizers.incrementAndGet(); + } + + public void decrementResizerAndNotify() { + int remaining = this.resizers.decrementAndGet(); + if (remaining == 0) { + synchronized (this) { + this.notifyAll(); + } + } + } + + public int getQueuePosition() { + return this.queuePosition.get(); + } + + public int subtractAndGetQueuePosition() { + return this.queuePosition.addAndGet(-QUEUE_INCREMENT); + } + + public void waitForAllResizers() { + if (this.resizers.get() > 0) { + for (int i = 0; i < 16; i++) { + if (this.resizers.get() == 0) { + break; + } + } + for (int i = 0; i < 16; i++) { + if (this.resizers.get() == 0) { + break; + } + Thread.yield(); + } + } + if (this.resizers.get() > 0) { + synchronized (this) { + while (this.resizers.get() > 0) { + try { + this.wait(); + } catch (InterruptedException e) { + // ignore + } + } + } + } + } + + public boolean isNotDone() { + return this.resizers.get() > 0; + } + + public void zeroOutQueuePosition() { + this.queuePosition.set(0); + } + } + + private static class Entry { + + final int key; + volatile int value; + volatile Entry next; + + /** + * 0 NORMAL + * 1 RESIZING + * 2 RESIZED + * 3 RESIZE_SENTINEL + * 4 RESIZE_CONTAINER + */ + final byte state; + + public Entry(int key, int value, byte state) { + this.key = key; + this.value = value; + this.state = state; + } + + public Entry(int key, int value) { + this.key = key; + this.value = value; + this.next = null; + this.state = 0; + } + + public Entry(int key, int value, Entry next) { + this.key = key; + this.value = value; + this.next = next; + this.state = 0; + } + + public int getKey() { + return key; + } + + public int getValue() { + return value; + } + + public Entry getNext() { + return next; + } + + @Override + public String toString() { + return this.key + "=" + this.value; + } + } + + /* ---------------- Iterator -------------- */ + + private static final class IteratorState { + private Entry[] currentTable; + private int start; + private int end; + + private IteratorState(Entry[] currentTable) { + this.currentTable = currentTable; + this.end = this.currentTable.length - 1; + } + + private IteratorState(Entry[] currentTable, int start, int end) { + this.currentTable = currentTable; + this.start = start; + this.end = end; + } + } + + /** + * The HashIterator class is an abstract base class for iterators over the map. + * It maintains the current state of the iteration, which includes the current table + * being iterated over and the index of the next entry to be returned. + * The findNext() method is used to advance the iterator to the next entry. + */ + private abstract class HashIterator implements IntIterator { + + private List todo; + private IteratorState currentState; + private Entry next; + private int index; + + protected HashIterator() { + this.currentState = new IteratorState(IntMapByDynamicHash.this.table); + this.findNext(); + } + + /** + * This method is used to advance the iterator to the next entry. + * It iterates over the entries in the current table from the current index + * until it finds a non-null entry. If it encounters a RESIZED or RESIZING entry, + * it helps with the resize operation and continues the iteration in the new table. + * If it reaches the end of the current table and there are still tables left to be + * iterated over, it switches to the next table. + */ + private void findNext() { + while (this.index < this.currentState.end) { + Entry o = + (Entry) IntMapByDynamicHash.tableAt(this.currentState.currentTable, this.index); + if (o == RESIZED || o == RESIZING) { + Entry[] nextArray = + IntMapByDynamicHash.this.helpWithResizeWhileCurrentIndex( + this.currentState.currentTable, this.index); + int endResized = this.index + 1; + while (endResized < this.currentState.end) { + if (IntMapByDynamicHash.tableAt(this.currentState.currentTable, + endResized) != RESIZED) { + break; + } + endResized++; + } + if (this.todo == null) { + this.todo = new ArrayList<>(4); + } + if (endResized < this.currentState.end) { + this.todo.add(new IteratorState( + this.currentState.currentTable, endResized, this.currentState.end)); + } + int powerTwoLength = this.currentState.currentTable.length - 1; + this.todo.add(new IteratorState(nextArray, this.index + powerTwoLength, + endResized + powerTwoLength)); + this.currentState.currentTable = nextArray; + this.currentState.end = endResized; + this.currentState.start = this.index; + } else if (o != null) { + this.next = o; + this.index++; + break; + } else { + this.index++; + } + } + if (this.next == null && this.index == this.currentState.end && this.todo != null && + !this.todo.isEmpty()) { + this.currentState = this.todo.remove(this.todo.size() - 1); + this.index = this.currentState.start; + this.findNext(); + } + } + + @Override + public final boolean hasNext() { + return this.next != null; + } + + final Entry nextEntry() { + Entry e = this.next; + if (e == null) { + throw new NoSuchElementException(); + } + + if ((this.next = e.getNext()) == null) { + this.findNext(); + } + return e; + } + } + + private final class ValueIterator extends HashIterator { + @Override + public int next() { + return this.nextEntry().getValue(); + } + } + + private final class KeyIterator extends HashIterator { + @Override + public int next() { + return this.nextEntry().getKey(); + } + } + + /* ---------------- Unsafe mechanics -------------- */ + private static final Unsafe UNSAFE = IntSet.UNSAFE; + private static final long ENTRY_ARRAY_BASE; + private static final int ENTRY_ARRAY_SHIFT; + private static final long INT_ARRAY_BASE; + private static final int INT_ARRAY_SHIFT; + private static final long SIZE_OFFSET; + + static { + try { + Class tableClass = Entry[].class; + ENTRY_ARRAY_BASE = UNSAFE.arrayBaseOffset(tableClass); + int objectArrayScale = UNSAFE.arrayIndexScale(tableClass); + if ((objectArrayScale & (objectArrayScale - 1)) != 0) { + throw new AssertionError("data type scale not a power of two"); + } + ENTRY_ARRAY_SHIFT = 31 - Integer.numberOfLeadingZeros(objectArrayScale); + + Class intArrayClass = int[].class; + INT_ARRAY_BASE = UNSAFE.arrayBaseOffset(intArrayClass); + int intArrayScale = UNSAFE.arrayIndexScale(intArrayClass); + if ((intArrayScale & (intArrayScale - 1)) != 0) { + throw new AssertionError("data type scale not a power of two"); + } + INT_ARRAY_SHIFT = 31 - Integer.numberOfLeadingZeros(intArrayScale); + + Class mapClass = IntMapByDynamicHash.class; + SIZE_OFFSET = UNSAFE.objectFieldOffset(mapClass.getDeclaredField("size")); + } catch (NoSuchFieldException | SecurityException e) { + throw new AssertionError(e); + } + } +} diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/util/collection/IntMapTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/util/collection/IntMapTest.java index 4e2ca9c388..29755bc718 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/util/collection/IntMapTest.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/unit/util/collection/IntMapTest.java @@ -21,17 +21,22 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; -import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - import org.apache.hugegraph.testutil.Assert; import org.apache.hugegraph.unit.BaseUnitTest; import org.apache.hugegraph.util.collection.IntIterator; import org.apache.hugegraph.util.collection.IntMap; +import org.apache.hugegraph.util.collection.IntMapByDynamicHash; +import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; public class IntMapTest extends BaseUnitTest { @@ -412,6 +417,106 @@ public void testIntFixedMapBySegmentsValuesWithMultiSegs() { } } + @Test + public void testIntMapByDynamicHashSingleThread() { + IntMap map = new IntMapByDynamicHash(); + int mapSize = 2000; + for (int i = 0; i < mapSize; i++) { + map.put(i, i + 1); + Assert.assertTrue(map.containsKey(i)); + Assert.assertFalse(map.containsKey(i + mapSize)); + Assert.assertEquals(i + 1, map.get(i)); + } + + for (int i = mapSize - 1; i >= 0; i--) { + map.put(i, i - 1); + Assert.assertTrue(map.containsKey(i)); + Assert.assertFalse(map.containsKey(i + mapSize)); + Assert.assertEquals(i - 1, map.get(i)); + } + + Assert.assertEquals(mapSize, map.size()); + map.clear(); + Assert.assertEquals(0, map.size()); + } + + @Test + public void testIntMapByDynamicHashMultiThread() throws InterruptedException { + IntMap map = new IntMapByDynamicHash(); + + //int cpus = IntSet.CPUS; + int cpus = 16; + ThreadPoolExecutor executor = + new ThreadPoolExecutor(cpus, cpus, 1, TimeUnit.MINUTES, + new LinkedBlockingDeque<>()) { + @Override + protected void afterExecute(Runnable r, Throwable t) { + super.afterExecute(r, t); + if (t != null) { + Assert.fail(t.getMessage()); + } + } + }; + ; + + AtomicInteger size = new AtomicInteger(); + int mapSize = 100; + CountDownLatch latch = new CountDownLatch(cpus); + for (int i = 1; i <= cpus; i++) { + int index = i; + executor.submit(() -> { + try { + for (int j = 0; j < mapSize; j++) { + int key = j + (index - 1) * mapSize; + map.put(key, j); + size.getAndIncrement(); + //Assert.assertTrue(map.containsKey(key)); + Assert.assertEquals(j, map.get(key)); + //System.out.println(key + " " + j); + } + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } finally { + latch.countDown(); + } + }); + } + + latch.await(); + System.out.println(); + + Assert.assertEquals(size.get(), map.size()); + } + + @Test + public void testIntMapByDynamicHashKeys() { + IntMap map = new IntMapByDynamicHash(); + for (int i = 0; i < 10000; i++) { + map.put(i, i + 100); + } + IntIterator iterator = map.keys(); + for (int i = 0; i < 10000; i++) { + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals(i, iterator.next()); + } + Assert.assertFalse(iterator.hasNext()); + } + + @Test + public void testIntMapByDynamicHashValues() { + IntMap map = new IntMapByDynamicHash(); + for (int i = 0; i < 10000; i++) { + map.put(i, i + 100); + } + IntIterator iterator = map.values(); + for (int i = 0; i < 10000; i++) { + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals(i + 100, iterator.next()); + } + Assert.assertFalse(iterator.hasNext()); + } + private IntMap fixed(int capacity) { return new IntMap.IntMapByFixedAddr(capacity); } diff --git a/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/BenchmarkConstants.java b/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/BenchmarkConstants.java index 1525e8143e..1641bc95c9 100644 --- a/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/BenchmarkConstants.java +++ b/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/BenchmarkConstants.java @@ -19,5 +19,5 @@ public class BenchmarkConstants { - public static String OUTPUT_PATH = "./hugegraph-test/target/"; + public static String OUTPUT_PATH = "./hugegraph-server/hugegraph-test/target/"; } diff --git a/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/map/MapRandomGetPutThroughputTest.java b/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/map/MapRandomGetPutThroughputTest.java index eafe4b861f..4423351b52 100644 --- a/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/map/MapRandomGetPutThroughputTest.java +++ b/hugegraph-server/hugegraph-test/src/test/java/org/apache/hugegraph/benchmark/map/MapRandomGetPutThroughputTest.java @@ -23,6 +23,7 @@ import org.apache.hugegraph.benchmark.BenchmarkConstants; import org.apache.hugegraph.benchmark.SimpleRandom; import org.apache.hugegraph.util.collection.IntMap; +import org.apache.hugegraph.util.collection.IntMapByDynamicHash; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -53,13 +54,15 @@ public class MapRandomGetPutThroughputTest { @Param(value = {"1000", "10000", "100000", "1000000"}) private int MAP_CAPACITY; - private ConcurrentHashMap concurrentHashMapNonCap; + private ConcurrentHashMap concurrentHashMapWithoutCap; - private ConcurrentHashMap concurrentHashMap; + private ConcurrentHashMap concurrentHashMapWithCap; - private IntMap.IntMapBySegments intMapBySegments; + private IntMap intMapBySegmentsWithCap; - private IntMap.IntMapByEcSegment intMapByEcSegments; + private IntMap intMapByDynamicHashWithoutCap; + + private IntMap intMapByDynamicHashWithCap; private static final int THREAD_COUNT = 8; @@ -67,10 +70,11 @@ public class MapRandomGetPutThroughputTest { @Setup(Level.Trial) public void prepareMap() { - this.concurrentHashMapNonCap = new ConcurrentHashMap<>(); - this.concurrentHashMap = new ConcurrentHashMap<>(MAP_CAPACITY); - this.intMapBySegments = new IntMap.IntMapBySegments(MAP_CAPACITY); - this.intMapByEcSegments = new IntMap.IntMapByEcSegment(); + this.concurrentHashMapWithoutCap = new ConcurrentHashMap<>(); + this.concurrentHashMapWithCap = new ConcurrentHashMap<>(MAP_CAPACITY); + this.intMapBySegmentsWithCap = new IntMap.IntMapBySegments(MAP_CAPACITY); + this.intMapByDynamicHashWithoutCap = new IntMapByDynamicHash(); + this.intMapByDynamicHashWithCap = new IntMapByDynamicHash(MAP_CAPACITY); } /** @@ -89,41 +93,51 @@ int next() { @Benchmark @Threads(THREAD_COUNT) public void randomGetPutOfConcurrentHashMapWithNoneInitCap(ThreadState state) { - int key = state.next() & (MAP_CAPACITY - 1); - if (!this.concurrentHashMapNonCap.containsKey(key)) { - this.concurrentHashMapNonCap.put(key, state.next()); + int key = state.next(); + if (!this.concurrentHashMapWithoutCap.containsKey(key)) { + this.concurrentHashMapWithoutCap.put(key, state.next()); } - this.concurrentHashMapNonCap.get(key); + this.concurrentHashMapWithoutCap.get(key); } @Benchmark @Threads(THREAD_COUNT) public void randomGetPutOfConcurrentHashMapWithInitCap(ThreadState state) { int key = state.next() & (MAP_CAPACITY - 1); - if (!this.concurrentHashMap.containsKey(key)) { - this.concurrentHashMap.put(key, state.next()); + if (!this.concurrentHashMapWithCap.containsKey(key)) { + this.concurrentHashMapWithCap.put(key, state.next()); } - this.concurrentHashMap.get(key); + this.concurrentHashMapWithCap.get(key); } @Benchmark @Threads(THREAD_COUNT) - public void randomGetPutOfIntMapBySegments(ThreadState state) { + public void randomGetPutOfIntMapBySegmentsWithInitCap(ThreadState state) { int key = state.next() & (MAP_CAPACITY - 1); - if (!this.intMapBySegments.containsKey(key)) { - this.intMapBySegments.put(key, state.next()); + if (!this.intMapBySegmentsWithCap.containsKey(key)) { + this.intMapBySegmentsWithCap.put(key, state.next()); + } + this.intMapBySegmentsWithCap.get(key); + } + + @Benchmark + @Threads(THREAD_COUNT) + public void randomGetPutOfIntMapByDynamicHashWithNoneCap(ThreadState state) { + int key = state.next(); + if (!this.intMapByDynamicHashWithoutCap.containsKey(key)) { + this.intMapByDynamicHashWithoutCap.put(key, state.next()); } - this.intMapBySegments.get(key); + this.intMapByDynamicHashWithoutCap.get(key); } @Benchmark @Threads(THREAD_COUNT) - public void randomGetPutOfIntMapByEcSegment(ThreadState state) { + public void randomGetPutOfIntMapByDynamicHashWithInitCap(ThreadState state) { int key = state.next() & (MAP_CAPACITY - 1); - if (!this.intMapByEcSegments.containsKey(key)) { - this.intMapByEcSegments.put(key, state.next()); + if (!this.intMapByDynamicHashWithCap.containsKey(key)) { + this.intMapByDynamicHashWithCap.put(key, state.next()); } - this.intMapByEcSegments.get(key); + this.intMapByDynamicHashWithCap.get(key); } public static void main(String[] args) throws RunnerException { From 2c6fcdc719a0afa7e2d1eaddb0c063d152445a8f Mon Sep 17 00:00:00 2001 From: Dandelion <49650772+aroundabout@users.noreply.github.com> Date: Wed, 13 Dec 2023 16:00:50 +0800 Subject: [PATCH 8/9] add maven args for stage or other args (#2386) --- hugegraph-server/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hugegraph-server/Dockerfile b/hugegraph-server/Dockerfile index a28e63ea13..47dc411c6e 100644 --- a/hugegraph-server/Dockerfile +++ b/hugegraph-server/Dockerfile @@ -21,7 +21,9 @@ FROM maven:3.9.0-eclipse-temurin-11 AS build COPY . /pkg WORKDIR /pkg -RUN mvn package -e -B -ntp -DskipTests -Dmaven.javadoc.skip=true && pwd && ls -l +ARG MAVEN_ARGS + +RUN mvn package $MAVEN_ARGS -e -B -ntp -DskipTests -Dmaven.javadoc.skip=true && pwd && ls -l # 2nd stage: runtime env FROM openjdk:11-slim From 4346b44f80b316eb9cb1aa5babafc90dfc71f2b1 Mon Sep 17 00:00:00 2001 From: vaughn Date: Thu, 14 Dec 2023 17:59:31 +0800 Subject: [PATCH 9/9] fix: TinkerPop unit test lack some lables (#2387) --- .../main/java/org/apache/hugegraph/tinkerpop/TestGraph.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java index 415e804626..f5c1ac7fb5 100644 --- a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java +++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/tinkerpop/TestGraph.java @@ -874,6 +874,10 @@ private void initBasicVertexLabelAndEdgeLabelExceptV(String defaultVL) { schema.indexLabel("bTOcByGremlinPartition").onE("bTOc") .by("gremlin.partitionGraphStrategy.partition") .ifNotExist().create(); + schema.edgeLabel("blah1").link(defaultVL, defaultVL) + .ifNotExist().create(); + schema.edgeLabel("blah2").link(defaultVL, defaultVL) + .ifNotExist().create(); } public void initEdgeLabelDefaultKnowsDefault(String defaultVL) {