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 6480d7f288..eb991c0f68 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
@@ -70,6 +70,8 @@
import org.apache.hugegraph.masterelection.RoleElectionStateMachine;
import org.apache.hugegraph.masterelection.StandardClusterRoleStore;
import org.apache.hugegraph.masterelection.StandardRoleElectionStateMachine;
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.util.RoundUtil;
import org.apache.hugegraph.meta.MetaManager;
import org.apache.hugegraph.perf.PerfUtil.Watched;
import org.apache.hugegraph.rpc.RpcServiceConfig4Client;
@@ -215,6 +217,13 @@ public StandardHugeGraph(HugeConfig config) {
this.readMode = GraphReadMode.OLTP_ONLY;
this.schedulerType = config.get(CoreOptions.SCHEDULER_TYPE);
+ MemoryManager.setMemoryMode(
+ MemoryManager.MemoryMode.fromValue(config.get(CoreOptions.MEMORY_MODE)));
+ MemoryManager.setMaxMemoryCapacityInBytes(config.get(CoreOptions.MAX_MEMORY_CAPACITY));
+ MemoryManager.setMaxMemoryCapacityForOneQuery(
+ config.get(CoreOptions.ONE_QUERY_MAX_MEMORY_CAPACITY));
+ RoundUtil.setAlignment(config.get(CoreOptions.MEMORY_ALIGNMENT));
+
LockUtil.init(this.name);
try {
@@ -477,8 +486,8 @@ private ISchemaTransaction openSchemaTransaction() throws HugeException {
try {
if (isHstore()) {
return new CachedSchemaTransactionV2(
- MetaManager.instance().metaDriver(),
- MetaManager.instance().cluster(), this.params);
+ MetaManager.instance().metaDriver(),
+ MetaManager.instance().cluster(), this.params);
}
return new CachedSchemaTransaction(this.params, loadSchemaStore());
} catch (BackendException e) {
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/cache/CachedBackendStore.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/cache/CachedBackendStore.java
index e266c0623f..3cc88bbf71 100644
--- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/cache/CachedBackendStore.java
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/cache/CachedBackendStore.java
@@ -215,16 +215,21 @@ public Number queryNumber(Query query) {
/**
* Query as an Id for cache
*/
- static class QueryId implements Id {
+ public static class QueryId implements Id {
- private String query;
- private int hashCode;
+ protected String query;
+ protected int hashCode;
public QueryId(Query q) {
this.query = q.toString();
this.hashCode = q.hashCode();
}
+ public QueryId(String query, int hashCode) {
+ this.query = query;
+ this.hashCode = hashCode;
+ }
+
@Override
public IdType type() {
return IdType.UNKNOWN;
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/EdgeId.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/EdgeId.java
index fe030a153f..14116d8575 100644
--- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/EdgeId.java
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/EdgeId.java
@@ -32,8 +32,8 @@
* > sortKeys > target-vertex-id }
* NOTE:
* 1. for edges with edgeLabelType = NORMAL: edgeLabelId = parentEdgeLabelId = subEdgeLabelId;
- * for edges with edgeLabelType = PARENT: edgeLabelId = subEdgeLabelId, parentEdgeLabelId =
- * edgeLabelId.fatherId
+ * for edges with edgeLabelType = PARENT: edgeLabelId = subEdgeLabelId, parentEdgeLabelId =
+ * edgeLabelId.fatherId
*
2.if we use `entry.type()` which is IN or OUT as a part of id,
* an edge's id will be different due to different directions (belongs
* to 2 owner vertex)
@@ -49,15 +49,14 @@ public class EdgeId implements Id {
HugeKeys.OTHER_VERTEX
};
- private final Id ownerVertexId;
- private final Directions direction;
- private final Id edgeLabelId;
- private final Id subLabelId;
- private final String sortValues;
- private final Id otherVertexId;
-
- private final boolean directed;
- private String cache;
+ protected final Id ownerVertexId;
+ protected final Id edgeLabelId;
+ protected final Id subLabelId;
+ protected final Id otherVertexId;
+ protected final Directions direction;
+ protected final boolean directed;
+ protected String sortValues;
+ protected String cache;
public EdgeId(HugeVertex ownerVertex, Directions direction,
Id edgeLabelId, Id subLabelId, String sortValues, HugeVertex otherVertex) {
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/IdGenerator.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/IdGenerator.java
index 9261d31fe8..17cc11684c 100644
--- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/IdGenerator.java
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/id/IdGenerator.java
@@ -119,15 +119,15 @@ public static IdType idType(Id id) {
return IdType.UNKNOWN;
}
- private static int compareType(Id id1, Id id2) {
+ public static int compareType(Id id1, Id id2) {
return idType(id1).ordinal() - idType(id2).ordinal();
}
/****************************** id defines ******************************/
- public static final class StringId implements Id {
+ public static class StringId implements Id {
- private final String id;
+ protected String id;
public StringId(String id) {
E.checkArgument(!id.isEmpty(), "The id can't be empty");
@@ -196,11 +196,11 @@ public String toString() {
}
}
- public static final class LongId extends Number implements Id {
+ public static class LongId extends Number implements Id {
private static final long serialVersionUID = -7732461469037400190L;
- private final long id;
+ protected Long id;
public LongId(long id) {
this.id = id;
@@ -270,7 +270,7 @@ public String toString() {
@Override
public int intValue() {
- return (int) this.id;
+ return this.id.intValue();
}
@Override
@@ -289,9 +289,9 @@ public double doubleValue() {
}
}
- public static final class UuidId implements Id {
+ public static class UuidId implements Id {
- private final UUID uuid;
+ protected UUID uuid;
public UuidId(String string) {
this(StringEncoding.uuid(string));
@@ -379,9 +379,9 @@ public String toString() {
/**
* This class is just used by backend store for wrapper object as Id
*/
- public static final class ObjectId implements Id {
+ public static class ObjectId implements Id {
- private final Object object;
+ protected Object object;
public ObjectId(Object object) {
E.checkNotNull(object, "object");
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/serializer/BinaryBackendEntry.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/serializer/BinaryBackendEntry.java
index 67e437d9cf..9dfb8b4e5c 100644
--- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/serializer/BinaryBackendEntry.java
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/backend/serializer/BinaryBackendEntry.java
@@ -208,10 +208,10 @@ public int hashCode() {
return this.id().hashCode() ^ this.columns.size();
}
- public static final class BinaryId implements Id {
+ public static class BinaryId implements Id {
- private final byte[] bytes;
- private final Id id;
+ protected byte[] bytes;
+ protected Id id;
public BinaryId(byte[] bytes, Id id) {
this.bytes = bytes;
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/config/CoreOptions.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/config/CoreOptions.java
index 21054902c2..2bfbedd2ae 100644
--- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/config/CoreOptions.java
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/config/CoreOptions.java
@@ -20,6 +20,7 @@
import static org.apache.hugegraph.backend.tx.GraphTransaction.COMMIT_BATCH;
import static org.apache.hugegraph.config.OptionChecker.allowValues;
import static org.apache.hugegraph.config.OptionChecker.disallowEmpty;
+import static org.apache.hugegraph.config.OptionChecker.nonNegativeInt;
import static org.apache.hugegraph.config.OptionChecker.positiveInt;
import static org.apache.hugegraph.config.OptionChecker.rangeInt;
@@ -681,4 +682,32 @@ public static synchronized CoreOptions instance() {
disallowEmpty(),
"127.0.0.1:8686"
);
+
+ public static final ConfigOption MEMORY_MODE = new ConfigOption<>(
+ "memory.mode",
+ "The memory mode used for query in HugeGraph.",
+ disallowEmpty(),
+ "off-heap"
+ );
+
+ public static final ConfigOption MAX_MEMORY_CAPACITY = new ConfigOption<>(
+ "memory.max_capacity",
+ "The maximum memory capacity that can be managed for all queries in HugeGraph.",
+ nonNegativeInt(),
+ Bytes.GB
+ );
+
+ public static final ConfigOption ONE_QUERY_MAX_MEMORY_CAPACITY = new ConfigOption<>(
+ "memory.one_query_max_capacity",
+ "The maximum memory capacity that can be managed for a query in HugeGraph.",
+ nonNegativeInt(),
+ Bytes.MB * 100
+ );
+
+ public static final ConfigOption MEMORY_ALIGNMENT = new ConfigOption<>(
+ "memory.alignment",
+ "The alignment used for round memory size.",
+ nonNegativeInt(),
+ 8L
+ );
}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/MemoryManager.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/MemoryManager.java
new file mode 100644
index 0000000000..b2ac195e3e
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/MemoryManager.java
@@ -0,0 +1,249 @@
+/*
+ * 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.memory;
+
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hugegraph.memory.arbitrator.MemoryArbitrator;
+import org.apache.hugegraph.memory.arbitrator.MemoryArbitratorImpl;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.pool.impl.QueryMemoryPool;
+import org.apache.hugegraph.memory.pool.impl.TaskMemoryPool;
+import org.apache.hugegraph.util.Bytes;
+import org.apache.hugegraph.util.ExecutorUtil;
+import org.jetbrains.annotations.TestOnly;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class supports memory management for HugeGraph queries.
+ *
+ * Memory management is divided into three levels: query level, task (thread) level, and operator
+ * level. For each new query, the Manager's {@code addQueryMemoryPool} method is called to
+ * construct a new queryPool. During query execution, newTaskPool and newOperatorPool are
+ * required on demand.
+ *
+ * Where memory needs to be requested, use {@code getCorrespondingTaskMemoryPool} to get the
+ * current taskPool, and use {@code getCurrentWorkingOperatorMemoryPool} to get the working
+ * OperatorPool from the taskPool, and use OperatorPool to request memory
+ *
+ * Note: current MemoryManager doesn't support on-heap management.
+ */
+public class MemoryManager {
+
+ private static final Logger LOG = LoggerFactory.getLogger(MemoryManager.class);
+ private static final int ARBITRATE_MEMORY_THREAD_NUM = 12;
+ private static final String QUERY_MEMORY_POOL_NAME_PREFIX = "QueryMemoryPool";
+ private static final String ARBITRATE_MEMORY_POOL_NAME = "ArbitrateMemoryPool";
+ public static final String DELIMINATOR = "_";
+
+ public static long MAX_MEMORY_CAPACITY_IN_BYTES = Bytes.GB;
+ public static long MAX_MEMORY_CAPACITY_FOR_ONE_QUERY = Bytes.MB * 100;
+ // Current available memory = MAX_MEMORY_CAPACITY - sum(allocated bytes)
+ private final AtomicLong currentAvailableMemoryInBytes =
+ new AtomicLong(MAX_MEMORY_CAPACITY_IN_BYTES);
+ private final AtomicLong currentOffHeapAllocatedMemoryInBytes = new AtomicLong(0);
+ private final AtomicLong currentOnHeapAllocatedMemoryInBytes = new AtomicLong(0);
+
+ private final Queue queryMemoryPools =
+ new PriorityQueue<>((o1, o2) -> (int) (o2.getFreeBytes() - o1.getFreeBytes()));
+ private final Map threadName2TaskMemoryPoolMap =
+ new ConcurrentHashMap<>();
+
+ private final MemoryArbitrator memoryArbitrator;
+ private final ExecutorService arbitrateExecutor;
+
+ private static MemoryMode MEMORY_MODE = MemoryMode.ENABLE_OFF_HEAP_MANAGEMENT;
+
+ private MemoryManager() {
+ this.memoryArbitrator = new MemoryArbitratorImpl(this);
+ // Since there is always only 1 working operator pool for 1 query, It is not possible to
+ // run local arbitration or global arbitration in parallel within a query. The thread
+ // pool here is to allow parallel arbitration between queries
+ this.arbitrateExecutor = ExecutorUtil.newFixedThreadPool(ARBITRATE_MEMORY_THREAD_NUM,
+ ARBITRATE_MEMORY_POOL_NAME);
+ }
+
+ public MemoryPool addQueryMemoryPool() {
+ int count = queryMemoryPools.size();
+ String poolName =
+ QUERY_MEMORY_POOL_NAME_PREFIX + DELIMINATOR + count + DELIMINATOR +
+ System.currentTimeMillis();
+ MemoryPool queryPool = new QueryMemoryPool(poolName, this);
+ queryMemoryPools.add(queryPool);
+ LOG.info("Manager added query memory pool {}", queryPool);
+ return queryPool;
+ }
+
+ public void gcQueryMemoryPool(MemoryPool pool) {
+ LOG.info("Manager gc query memory pool {}", pool);
+ queryMemoryPools.remove(pool);
+ pool.releaseSelf(String.format("GC query memory pool %s", pool), false);
+ }
+
+ public void returnReclaimedTaskMemory(long bytes) {
+ currentAvailableMemoryInBytes.addAndGet(bytes);
+ }
+
+ public void consumeAvailableMemory(long size) {
+ currentAvailableMemoryInBytes.addAndGet(-size);
+ }
+
+ public long triggerLocalArbitration(MemoryPool targetPool, long neededBytes,
+ MemoryPool requestPool) {
+ LOG.info("LocalArbitration triggered by {}: needed bytes={}", targetPool, neededBytes);
+ Future future =
+ arbitrateExecutor.submit(
+ () -> memoryArbitrator.reclaimLocally(targetPool, neededBytes,
+ requestPool));
+ try {
+ return future.get(MemoryArbitrator.MAX_WAIT_TIME_FOR_LOCAL_RECLAIM,
+ TimeUnit.MILLISECONDS);
+ } catch (TimeoutException e) {
+ LOG.warn("MemoryManager: arbitration locally for {} timed out", targetPool, e);
+ } catch (InterruptedException | ExecutionException e) {
+ LOG.error("MemoryManager: arbitration locally for {} interrupted or failed",
+ targetPool,
+ e);
+ }
+ return 0;
+ }
+
+ public long triggerGlobalArbitration(MemoryPool requestPool, long neededBytes) {
+ LOG.info("GlobalArbitration triggered by {}: needed bytes={}", requestPool, neededBytes);
+ Future future =
+ arbitrateExecutor.submit(
+ () -> memoryArbitrator.reclaimGlobally(requestPool, neededBytes));
+ try {
+ return future.get(MemoryArbitrator.MAX_WAIT_TIME_FOR_GLOBAL_RECLAIM,
+ TimeUnit.MILLISECONDS);
+ } catch (TimeoutException e) {
+ LOG.warn("MemoryManager: arbitration globally for {} timed out", requestPool, e);
+ } catch (InterruptedException | ExecutionException e) {
+ LOG.error("MemoryManager: arbitration globally for {} interrupted or failed",
+ requestPool, e);
+ }
+ return 0;
+ }
+
+ public synchronized long handleRequestFromQueryPool(long size, String action) {
+ if (currentAvailableMemoryInBytes.get() < size) {
+ LOG.info("There isn't enough memory for query pool to {}: " +
+ "requestSize={}, remainingCapacity={}", size, action,
+ currentAvailableMemoryInBytes.get());
+ return -1;
+ }
+ return size;
+ }
+
+ /**
+ * Used by task thread to find its memory pool to release self's memory resource when exiting.
+ */
+ public MemoryPool getCorrespondingTaskMemoryPool(String threadName) {
+ return threadName2TaskMemoryPoolMap.getOrDefault(threadName, null);
+ }
+
+ public void bindCorrespondingTaskMemoryPool(String threadName, TaskMemoryPool memoryPool) {
+ threadName2TaskMemoryPoolMap.computeIfAbsent(threadName, key -> memoryPool);
+ }
+
+ public void removeCorrespondingTaskMemoryPool(String threadName) {
+ threadName2TaskMemoryPoolMap.remove(threadName);
+ }
+
+ public Queue getCurrentQueryMemoryPools() {
+ return new PriorityQueue<>(queryMemoryPools);
+ }
+
+ public AtomicLong getCurrentOnHeapAllocatedMemoryInBytes() {
+ return currentOnHeapAllocatedMemoryInBytes;
+ }
+
+ public AtomicLong getCurrentOffHeapAllocatedMemoryInBytes() {
+ return currentOffHeapAllocatedMemoryInBytes;
+ }
+
+ public static void setMemoryMode(MemoryMode conf) {
+ MEMORY_MODE = conf;
+ }
+
+ public static MemoryMode getMemoryMode() {
+ return MEMORY_MODE;
+ }
+
+ public static void setMaxMemoryCapacityInBytes(long maxMemoryCapacityInBytes) {
+ MAX_MEMORY_CAPACITY_IN_BYTES = maxMemoryCapacityInBytes;
+ }
+
+ public static void setMaxMemoryCapacityForOneQuery(long maxMemoryCapacityForOneQuery) {
+ MAX_MEMORY_CAPACITY_FOR_ONE_QUERY = maxMemoryCapacityForOneQuery;
+ }
+
+ @TestOnly
+ public AtomicLong getCurrentAvailableMemoryInBytes() {
+ return currentAvailableMemoryInBytes;
+ }
+
+ private static class MemoryManagerHolder {
+
+ private static final MemoryManager INSTANCE = new MemoryManager();
+
+ private MemoryManagerHolder() {
+ // empty constructor
+ }
+ }
+
+ public static MemoryManager getInstance() {
+ return MemoryManagerHolder.INSTANCE;
+ }
+
+ public enum MemoryMode {
+ ENABLE_OFF_HEAP_MANAGEMENT("off-heap"),
+ ENABLE_ON_HEAP_MANAGEMENT("on-heap"),
+ DISABLE_MEMORY_MANAGEMENT("disable");
+
+ private final String value;
+
+ MemoryMode(String value) {
+ this.value = value;
+ }
+
+ public String getValue() {
+ return value;
+ }
+
+ public static MemoryMode fromValue(String value) {
+ if (value.equalsIgnoreCase(ENABLE_ON_HEAP_MANAGEMENT.getValue())) {
+ return ENABLE_ON_HEAP_MANAGEMENT;
+ } else if (value.equalsIgnoreCase(ENABLE_OFF_HEAP_MANAGEMENT.getValue())) {
+ return ENABLE_OFF_HEAP_MANAGEMENT;
+ }
+ // return DISABLE by default
+ return DISABLE_MEMORY_MANAGEMENT;
+ }
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/MemoryAllocator.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/MemoryAllocator.java
new file mode 100644
index 0000000000..9e8f443429
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/MemoryAllocator.java
@@ -0,0 +1,29 @@
+/*
+ * 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.memory.allocator;
+
+public interface MemoryAllocator {
+
+ Object tryToAllocate(long size);
+
+ Object forceAllocate(long size);
+
+ void returnMemoryToManager(long size);
+
+ void releaseMemoryBlock(Object memoryBlock);
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/NettyMemoryAllocator.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/NettyMemoryAllocator.java
new file mode 100644
index 0000000000..8a825ac159
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/NettyMemoryAllocator.java
@@ -0,0 +1,70 @@
+/*
+ * 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.memory.allocator;
+
+import org.apache.hugegraph.memory.MemoryManager;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.util.ReferenceCountUtil;
+
+/**
+ * This class makes fully use of Netty's efficient memory management strategy.
+ */
+public class NettyMemoryAllocator implements MemoryAllocator {
+
+ private final PooledByteBufAllocator offHeapAllocator = PooledByteBufAllocator.DEFAULT;
+ private final MemoryManager memoryManager;
+
+ public NettyMemoryAllocator(MemoryManager memoryManager) {
+ this.memoryManager = memoryManager;
+ }
+
+ @Override
+ public ByteBuf forceAllocate(long size) {
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().addAndGet(size);
+ return offHeapAllocator.directBuffer((int) size);
+ }
+
+ @Override
+ public ByteBuf tryToAllocate(long size) {
+ if (memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().get() +
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get() + size <
+ MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES) {
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().addAndGet(size);
+ return offHeapAllocator.directBuffer((int) size);
+ }
+ return null;
+ }
+
+ @Override
+ public void returnMemoryToManager(long size) {
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().addAndGet(-size);
+ }
+
+ @Override
+ public void releaseMemoryBlock(Object memoryBlock) {
+ if (!(memoryBlock instanceof ByteBuf)) {
+ throw new IllegalArgumentException(
+ String.format("memoryBlock must be ByteBuf, current " +
+ "class: %s", memoryBlock.getClass()));
+ }
+ ByteBuf buf = (ByteBuf) memoryBlock;
+ ReferenceCountUtil.safeRelease(buf);
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/OnHeapMemoryAllocator.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/OnHeapMemoryAllocator.java
new file mode 100644
index 0000000000..3ed229b417
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/allocator/OnHeapMemoryAllocator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.memory.allocator;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hugegraph.memory.MemoryManager;
+
+public class OnHeapMemoryAllocator implements MemoryAllocator {
+
+ private final MemoryManager memoryManager;
+
+ public OnHeapMemoryAllocator(MemoryManager memoryManager) {
+ this.memoryManager = memoryManager;
+ }
+
+ @Override
+ public AtomicReference tryToAllocate(long size) {
+ if (memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().get() +
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get() + size <
+ MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES) {
+ memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().addAndGet(size);
+ byte[] memoryBlock = new byte[(int) size];
+ return new AtomicReference<>(memoryBlock);
+ }
+ return null;
+ }
+
+ @Override
+ public AtomicReference forceAllocate(long size) {
+ memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().addAndGet(size);
+ byte[] memoryBlock = new byte[(int) size];
+ return new AtomicReference<>(memoryBlock);
+ }
+
+ @Override
+ public void returnMemoryToManager(long size) {
+ memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().addAndGet(-size);
+ }
+
+ @Override
+ public void releaseMemoryBlock(Object memoryBlock) {
+ ((AtomicReference) memoryBlock).set(null);
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/arbitrator/MemoryArbitrator.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/arbitrator/MemoryArbitrator.java
new file mode 100644
index 0000000000..4cb2fe55a6
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/arbitrator/MemoryArbitrator.java
@@ -0,0 +1,31 @@
+/*
+ * 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.memory.arbitrator;
+
+import org.apache.hugegraph.memory.pool.MemoryPool;
+
+public interface MemoryArbitrator {
+
+ long MAX_WAIT_TIME_FOR_LOCAL_RECLAIM = 2000;
+
+ long MAX_WAIT_TIME_FOR_GLOBAL_RECLAIM = 5000;
+
+ long reclaimLocally(MemoryPool queryPool, long neededBytes, MemoryPool requestingPool);
+
+ long reclaimGlobally(MemoryPool queryPool, long neededBytes);
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/arbitrator/MemoryArbitratorImpl.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/arbitrator/MemoryArbitratorImpl.java
new file mode 100644
index 0000000000..d8ce73afde
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/arbitrator/MemoryArbitratorImpl.java
@@ -0,0 +1,73 @@
+/*
+ * 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.memory.arbitrator;
+
+import java.util.Queue;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MemoryArbitratorImpl implements MemoryArbitrator {
+
+ private static final Logger LOG = LoggerFactory.getLogger(MemoryArbitratorImpl.class);
+ private final MemoryManager memoryManager;
+
+ public MemoryArbitratorImpl(MemoryManager memoryManager) {
+ this.memoryManager = memoryManager;
+ }
+
+ @Override
+ public long reclaimLocally(MemoryPool queryPool, long neededBytes, MemoryPool requestingPool) {
+ long startTime = System.currentTimeMillis();
+ long res = queryPool.tryToReclaimLocalMemory(neededBytes, requestingPool);
+ LOG.info("[{}] reclaim local memory: {} bytes, took {} ms",
+ queryPool,
+ res,
+ System.currentTimeMillis() - startTime);
+ return res;
+ }
+
+ @Override
+ public long reclaimGlobally(MemoryPool queryPool, long neededBytes) {
+ long startTime = System.currentTimeMillis();
+ long totalReclaimedBytes = 0;
+ long currentNeededBytes = neededBytes;
+ Queue currentMemoryPool = this.memoryManager.getCurrentQueryMemoryPools();
+ while (!currentMemoryPool.isEmpty()) {
+ MemoryPool memoryPool = currentMemoryPool.poll();
+ if (memoryPool.equals(queryPool)) {
+ continue;
+ }
+ LOG.info("Global reclaim triggerred by {} select {} to reclaim", queryPool,
+ memoryPool);
+ long res = memoryPool.tryToReclaimLocalMemory(currentNeededBytes, queryPool);
+ totalReclaimedBytes += res;
+ currentNeededBytes -= res;
+ if (currentNeededBytes <= 0) {
+ break;
+ }
+ }
+ LOG.info("[{}] reclaim global memory: {} bytes, took {} ms",
+ queryPool,
+ totalReclaimedBytes,
+ System.currentTimeMillis() - startTime);
+ return totalReclaimedBytes;
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/OffHeapObject.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/OffHeapObject.java
new file mode 100644
index 0000000000..d192e8d928
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/OffHeapObject.java
@@ -0,0 +1,57 @@
+/*
+ * 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.memory.consumer;
+
+import java.util.List;
+
+import org.apache.hugegraph.memory.pool.MemoryPool;
+
+import io.netty.buffer.ByteBuf;
+
+/**
+ * This interface is used by immutable, memory-heavy objects which will be stored in off heap.
+ */
+public interface OffHeapObject {
+
+ /**
+ * This method will read from off-heap ByteBuf storing binary data of self.
+ * Note: need class cast by hand when using.
+ *
+ * @return self value
+ */
+ Object zeroCopyReadFromByteBuf();
+
+ /**
+ * Serialize to DataOutputStream in stack first, then request an off heap ByteBuf from
+ * OperatorMemoryPool based on size of DataOutputStream. Finally, serializing it to ByteBuf.
+ */
+ void serializeSelfToByteBuf(MemoryPool memoryPool);
+
+ /**
+ * Called after serializingSelfToByteBuf, pointing all self's on heap vars to null, in order
+ * to let GC release all its on heap memory.
+ */
+ void releaseOriginalVarsOnHeap();
+
+ /**
+ * Called by memoryPool to release all its holding memory block when memoryPool release self.
+ *
+ * @return all holding memory block allocated by memoryPool
+ */
+ List getAllMemoryBlock();
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/factory/IdFactory.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/factory/IdFactory.java
new file mode 100644
index 0000000000..976b9cb373
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/factory/IdFactory.java
@@ -0,0 +1,328 @@
+/*
+ * 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.memory.consumer.factory;
+
+import java.util.UUID;
+
+import org.apache.hugegraph.backend.cache.CachedBackendStore;
+import org.apache.hugegraph.backend.id.EdgeId;
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.id.IdGenerator;
+import org.apache.hugegraph.backend.query.Query;
+import org.apache.hugegraph.backend.serializer.BinaryBackendEntry;
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.consumer.impl.id.BinaryIdOffHeap;
+import org.apache.hugegraph.memory.consumer.impl.id.EdgeIdOffHeap;
+import org.apache.hugegraph.memory.consumer.impl.id.LongIdOffHeap;
+import org.apache.hugegraph.memory.consumer.impl.id.ObjectIdOffHeap;
+import org.apache.hugegraph.memory.consumer.impl.id.QueryIdOffHeap;
+import org.apache.hugegraph.memory.consumer.impl.id.StringIdOffHeap;
+import org.apache.hugegraph.memory.consumer.impl.id.UuidIdOffHeap;
+import org.apache.hugegraph.memory.pool.impl.TaskMemoryPool;
+import org.apache.hugegraph.structure.HugeVertex;
+import org.apache.hugegraph.type.define.Directions;
+
+// NOTE: current MemoryManager doesn't support on-heap management.
+public class IdFactory {
+
+ private MemoryManager.MemoryMode memoryMode;
+
+ private IdFactory() {
+ // empty constructor
+ }
+
+ /**
+ * If using off-heap mode, param id must be OffHeapObject
+ */
+ public BinaryBackendEntry.BinaryId newBinaryId(byte[] bytes, Id id) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new BinaryIdOffHeap(bytes, null,
+ taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ (OffHeapObject) id);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new BinaryBackendEntry.BinaryId(bytes, id);
+ }
+ }
+
+ public IdGenerator.LongId newLongId(long id) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new LongIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ id);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.LongId(id);
+ }
+ }
+
+ public IdGenerator.LongId newLongId(byte[] bytes) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new LongIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ bytes);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.LongId(bytes);
+ }
+ }
+
+ public IdGenerator.ObjectId newObjectId(Object object) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new ObjectIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ object);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.ObjectId(object);
+ }
+ }
+
+ public CachedBackendStore.QueryId newQueryId(Query q) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new QueryIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ q);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new CachedBackendStore.QueryId(q);
+ }
+ }
+
+ public IdGenerator.StringId newStringId(String id) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new StringIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ id);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.StringId(id);
+ }
+ }
+
+ public IdGenerator.StringId newStringId(byte[] bytes) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new StringIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ bytes);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.StringId(bytes);
+ }
+ }
+
+ public IdGenerator.UuidId newUuidId(String id) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new UuidIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ id);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.UuidId(id);
+ }
+ }
+
+ public IdGenerator.UuidId newUuidId(byte[] bytes) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new UuidIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ bytes);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.UuidId(bytes);
+ }
+ }
+
+ public IdGenerator.UuidId newUuidId(UUID id) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new UuidIdOffHeap(taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ id);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new IdGenerator.UuidId(id);
+ }
+ }
+
+ public EdgeId newEdgeId(HugeVertex ownerVertex,
+ Directions direction,
+ Id edgeLabelId,
+ Id subLabelId,
+ String sortValues,
+ HugeVertex otherVertex) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new EdgeIdOffHeap(ownerVertex,
+ direction,
+ null,
+ null,
+ sortValues,
+ otherVertex,
+ taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ (OffHeapObject) ownerVertex.id(),
+ (OffHeapObject) edgeLabelId,
+ (OffHeapObject) subLabelId,
+ (OffHeapObject) otherVertex.id()
+ );
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new EdgeId(ownerVertex, direction, edgeLabelId, subLabelId, sortValues,
+ otherVertex);
+ }
+ }
+
+ public EdgeId newEdgeId(Id ownerVertexId,
+ Directions direction,
+ Id edgeLabelId,
+ Id subLabelId,
+ String sortValues,
+ Id otherVertexId) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new EdgeIdOffHeap((Id) null,
+ direction,
+ null,
+ null,
+ sortValues,
+ null,
+ taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ (OffHeapObject) ownerVertexId,
+ (OffHeapObject) edgeLabelId,
+ (OffHeapObject) subLabelId,
+ (OffHeapObject) otherVertexId
+ );
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new EdgeId(ownerVertexId, direction, edgeLabelId, subLabelId, sortValues,
+ otherVertexId);
+ }
+ }
+
+ public EdgeId newEdgeId(Id ownerVertexId,
+ Directions direction,
+ Id edgeLabelId,
+ Id subLabelId,
+ String sortValues,
+ Id otherVertexId,
+ boolean directed) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new EdgeIdOffHeap(null,
+ direction,
+ null,
+ null,
+ sortValues,
+ null,
+ directed,
+ taskMemoryPool.getCurrentWorkingOperatorMemoryPool(),
+ (OffHeapObject) ownerVertexId,
+ (OffHeapObject) edgeLabelId,
+ (OffHeapObject) subLabelId,
+ (OffHeapObject) otherVertexId
+ );
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new EdgeId(ownerVertexId, direction, edgeLabelId, subLabelId, sortValues,
+ otherVertexId, directed);
+ }
+ }
+
+ private static class IdFactoryHolder {
+
+ private static final IdFactory INSTANCE = new IdFactory();
+
+ private IdFactoryHolder() {
+ // empty constructor
+ }
+ }
+
+ public static IdFactory getInstance() {
+ IdFactory instance = IdFactoryHolder.INSTANCE;
+ if (instance.memoryMode == null) {
+ instance.memoryMode = MemoryManager.getMemoryMode();
+ }
+ return IdFactoryHolder.INSTANCE;
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/factory/PropertyFactory.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/factory/PropertyFactory.java
new file mode 100644
index 0000000000..c628155eb1
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/factory/PropertyFactory.java
@@ -0,0 +1,94 @@
+/*
+ * 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.memory.consumer.factory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.consumer.impl.property.HugeEdgePropertyOffHeap;
+import org.apache.hugegraph.memory.consumer.impl.property.HugeVertexPropertyOffHeap;
+import org.apache.hugegraph.memory.pool.impl.TaskMemoryPool;
+import org.apache.hugegraph.schema.PropertyKey;
+import org.apache.hugegraph.structure.HugeEdgeProperty;
+import org.apache.hugegraph.structure.HugeElement;
+import org.apache.hugegraph.structure.HugeVertexProperty;
+
+// NOTE: current MemoryManager doesn't support on-heap management.
+public class PropertyFactory {
+
+ private MemoryManager.MemoryMode memoryMode;
+
+ private PropertyFactory() {
+ // empty constructor
+ }
+
+ public HugeEdgeProperty newHugeEdgeProperty(HugeElement owner, PropertyKey key,
+ V value) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new HugeEdgePropertyOffHeap<>(
+ taskMemoryPool.getCurrentWorkingOperatorMemoryPool(), owner, key, value);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new HugeEdgeProperty<>(owner, key, value);
+ }
+ }
+
+ public HugeVertexProperty newHugeVertexProperty(HugeElement owner, PropertyKey key,
+ V value) {
+ switch (memoryMode) {
+ case ENABLE_ON_HEAP_MANAGEMENT:
+ case ENABLE_OFF_HEAP_MANAGEMENT:
+ TaskMemoryPool taskMemoryPool = (TaskMemoryPool) MemoryManager.getInstance()
+ .getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ return new HugeVertexPropertyOffHeap<>(
+ taskMemoryPool.getCurrentWorkingOperatorMemoryPool(), owner, key, value);
+ case DISABLE_MEMORY_MANAGEMENT:
+ default:
+ return new HugeVertexProperty<>(owner, key, value);
+ }
+ }
+
+ private static class PropertyFactoryHolder {
+
+ private static final Map, PropertyFactory>> FACTORIES_MAP =
+ new ConcurrentHashMap<>();
+
+ private PropertyFactoryHolder() {
+ // empty constructor
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public static PropertyFactory getInstance(Class clazz) {
+ PropertyFactory instance = (PropertyFactory) PropertyFactoryHolder.FACTORIES_MAP
+ .computeIfAbsent(clazz, k -> new PropertyFactory<>());
+ if (instance.memoryMode == null) {
+ instance.memoryMode = MemoryManager.getMemoryMode();
+ }
+ return instance;
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/BinaryIdOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/BinaryIdOffHeap.java
new file mode 100644
index 0000000000..e6444f3061
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/BinaryIdOffHeap.java
@@ -0,0 +1,134 @@
+/*
+ * 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.memory.consumer.impl.id;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.serializer.BinaryBackendEntry;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.util.Bytes;
+import org.apache.hugegraph.util.E;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+
+public class BinaryIdOffHeap extends BinaryBackendEntry.BinaryId implements OffHeapObject {
+
+ private final OffHeapObject originId;
+ private ByteBuf bytesOffHeap;
+
+ public BinaryIdOffHeap(byte[] bytes, Id id, MemoryPool memoryPool, OffHeapObject originId) {
+ super(bytes, id);
+ this.originId = originId;
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ this.bytesOffHeap = (ByteBuf) memoryPool.requireMemory(bytes.length, memoryPool);
+ this.bytesOffHeap.markReaderIndex();
+ this.bytesOffHeap.writeBytes(bytes);
+ }
+
+ @Override
+ public BinaryBackendEntry.BinaryId zeroCopyReadFromByteBuf() {
+ return new BinaryBackendEntry.BinaryId(ByteBufUtil.getBytes(bytesOffHeap),
+ (Id) originId.zeroCopyReadFromByteBuf());
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(bytesOffHeap);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.bytes = null;
+ this.id = null;
+ }
+
+ @Override
+ public Object asObject() {
+ return bytesOffHeap.nioBuffer();
+ }
+
+ @Override
+ public String toString() {
+ return "0x" + Bytes.toHex(asBytes());
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (!(other instanceof BinaryIdOffHeap)) {
+ return false;
+ }
+ return bytesOffHeap.equals(((BinaryIdOffHeap) other).bytesOffHeap);
+ }
+
+ @Override
+ public int hashCode() {
+ return bytesOffHeap.hashCode();
+ }
+
+ @Override
+ public int length() {
+ return bytesOffHeap.readableBytes();
+ }
+
+ @Override
+ public byte[] asBytes(int offset) {
+ E.checkArgument(offset < this.bytesOffHeap.readableBytes(),
+ "Invalid offset %s, must be < length %s",
+ offset, this.bytesOffHeap.readableBytes());
+ try {
+ // zero-copy read
+ byte[] tmpBytes = new byte[offset];
+ this.bytesOffHeap.readBytes(tmpBytes);
+ return tmpBytes;
+ } finally {
+ this.bytesOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public byte[] asBytes() {
+ try {
+ // zero-copy read
+ byte[] tmpBytes = new byte[bytesOffHeap.readableBytes()];
+ this.bytesOffHeap.readBytes(tmpBytes);
+ return tmpBytes;
+ } finally {
+ this.bytesOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public int compareTo(Id other) {
+ return bytesOffHeap.compareTo(((BinaryIdOffHeap) other).bytesOffHeap);
+ }
+
+ @Override
+ public Id origin() {
+ return (Id) originId.zeroCopyReadFromByteBuf();
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/EdgeIdOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/EdgeIdOffHeap.java
new file mode 100644
index 0000000000..b77de337a8
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/EdgeIdOffHeap.java
@@ -0,0 +1,286 @@
+/*
+ * 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.memory.consumer.impl.id;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hugegraph.backend.id.EdgeId;
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.id.IdUtil;
+import org.apache.hugegraph.backend.id.SplicingIdGenerator;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.structure.HugeVertex;
+import org.apache.hugegraph.type.define.Directions;
+
+import com.google.common.collect.Lists;
+
+import io.netty.buffer.ByteBuf;
+
+// TODO: rewrite static method in EdgeId
+public class EdgeIdOffHeap extends EdgeId implements OffHeapObject {
+
+ private final MemoryPool memoryPool;
+ private final OffHeapObject ownerVertexIdOffHeap;
+ private final OffHeapObject edgeLabelIdOffHeap;
+ private final OffHeapObject subLabelIdOffHeap;
+ private final OffHeapObject otherVertexIdOffHeap;
+ private ByteBuf sortValuesOffHeap;
+ private ByteBuf cacheOffHeap;
+
+ public EdgeIdOffHeap(HugeVertex ownerVertex,
+ Directions direction,
+ Id edgeLabelId,
+ Id subLabelId,
+ String sortValues,
+ HugeVertex otherVertex,
+ MemoryPool memoryPool,
+ OffHeapObject ownerVertexIdOffHeap,
+ OffHeapObject edgeLabelIdOffHeap,
+ OffHeapObject subLabelIdOffHeap,
+ OffHeapObject otherVertexIdOffHeap) {
+ super(ownerVertex, direction, edgeLabelId, subLabelId, sortValues, otherVertex);
+ this.memoryPool = memoryPool;
+ this.ownerVertexIdOffHeap = ownerVertexIdOffHeap;
+ this.edgeLabelIdOffHeap = edgeLabelIdOffHeap;
+ this.subLabelIdOffHeap = subLabelIdOffHeap;
+ this.otherVertexIdOffHeap = otherVertexIdOffHeap;
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ public EdgeIdOffHeap(Id ownerVertexId,
+ Directions direction,
+ Id edgeLabelId,
+ Id subLabelId,
+ String sortValues,
+ Id otherVertexId,
+ MemoryPool memoryPool,
+ OffHeapObject ownerVertexIdOffHeap,
+ OffHeapObject edgeLabelIdOffHeap,
+ OffHeapObject subLabelIdOffHeap,
+ OffHeapObject otherVertexIdOffHeap) {
+ super(ownerVertexId, direction, edgeLabelId, subLabelId,
+ sortValues, otherVertexId, false);
+ this.memoryPool = memoryPool;
+ this.ownerVertexIdOffHeap = ownerVertexIdOffHeap;
+ this.edgeLabelIdOffHeap = edgeLabelIdOffHeap;
+ this.subLabelIdOffHeap = subLabelIdOffHeap;
+ this.otherVertexIdOffHeap = otherVertexIdOffHeap;
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ public EdgeIdOffHeap(Id ownerVertexId,
+ Directions direction,
+ Id edgeLabelId,
+ Id subLabelId,
+ String sortValues,
+ Id otherVertexId,
+ boolean directed,
+ MemoryPool memoryPool,
+ OffHeapObject ownerVertexIdOffHeap,
+ OffHeapObject edgeLabelIdOffHeap,
+ OffHeapObject subLabelIdOffHeap,
+ OffHeapObject otherVertexIdOffHeap) {
+ super(ownerVertexId, direction, edgeLabelId, subLabelId, sortValues, otherVertexId,
+ directed);
+ this.memoryPool = memoryPool;
+ this.ownerVertexIdOffHeap = ownerVertexIdOffHeap;
+ this.edgeLabelIdOffHeap = edgeLabelIdOffHeap;
+ this.subLabelIdOffHeap = subLabelIdOffHeap;
+ this.otherVertexIdOffHeap = otherVertexIdOffHeap;
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ try {
+ return new EdgeId((Id) this.ownerVertexIdOffHeap.zeroCopyReadFromByteBuf(),
+ this.direction,
+ (Id) this.edgeLabelIdOffHeap.zeroCopyReadFromByteBuf(),
+ (Id) this.subLabelIdOffHeap.zeroCopyReadFromByteBuf(),
+ this.sortValuesOffHeap.toString(StandardCharsets.UTF_8),
+ (Id) this.otherVertexIdOffHeap.zeroCopyReadFromByteBuf());
+ } finally {
+ this.sortValuesOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ byte[] stringBytes = sortValues.getBytes((StandardCharsets.UTF_8));
+ this.sortValuesOffHeap =
+ (ByteBuf) this.memoryPool.requireMemory(stringBytes.length, memoryPool);
+ this.sortValuesOffHeap.markReaderIndex();
+ this.sortValuesOffHeap.writeBytes(stringBytes);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.sortValues = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return this.cacheOffHeap == null ? Collections.singletonList(this.sortValuesOffHeap) :
+ Lists.newArrayList(this.sortValuesOffHeap,
+ this.cacheOffHeap);
+ }
+
+ @Override
+ public EdgeId switchDirection() {
+ Directions newDirection = this.direction.opposite();
+ return new EdgeIdOffHeap(this.otherVertexId,
+ newDirection,
+ this.edgeLabelId,
+ this.subLabelId,
+ this.sortValues,
+ this.ownerVertexId,
+ this.memoryPool,
+ this.ownerVertexIdOffHeap,
+ this.edgeLabelIdOffHeap,
+ this.subLabelIdOffHeap,
+ this.otherVertexIdOffHeap);
+ }
+
+ @Override
+ public EdgeId directed(boolean directed) {
+ return new EdgeIdOffHeap(this.otherVertexId,
+ this.direction,
+ this.edgeLabelId,
+ this.subLabelId,
+ this.sortValues,
+ this.ownerVertexId,
+ directed,
+ this.memoryPool,
+ this.ownerVertexIdOffHeap,
+ this.edgeLabelIdOffHeap,
+ this.subLabelIdOffHeap,
+ this.otherVertexIdOffHeap);
+ }
+
+ @Override
+ public Id ownerVertexId() {
+ return (Id) this.ownerVertexIdOffHeap.zeroCopyReadFromByteBuf();
+ }
+
+ @Override
+ public Id edgeLabelId() {
+ return (Id) this.edgeLabelIdOffHeap.zeroCopyReadFromByteBuf();
+ }
+
+ @Override
+ public Id subLabelId() {
+ return (Id) this.subLabelIdOffHeap.zeroCopyReadFromByteBuf();
+ }
+
+ @Override
+ public String sortValues() {
+ try {
+ return this.sortValuesOffHeap.toString(StandardCharsets.UTF_8);
+ } finally {
+ this.sortValuesOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public Id otherVertexId() {
+ return (Id) this.otherVertexIdOffHeap.zeroCopyReadFromByteBuf();
+ }
+
+ @Override
+ public String asString() {
+ if (this.cacheOffHeap != null) {
+ try {
+ return this.cacheOffHeap.toString(StandardCharsets.UTF_8);
+ } finally {
+ this.cacheOffHeap.resetReaderIndex();
+ }
+ }
+ String tmpCache;
+ if (this.directed) {
+ tmpCache = SplicingIdGenerator.concat(
+ IdUtil.writeString((Id) this.ownerVertexIdOffHeap.zeroCopyReadFromByteBuf()),
+ this.direction.type().string(),
+ IdUtil.writeLong((Id) this.edgeLabelIdOffHeap.zeroCopyReadFromByteBuf()),
+ IdUtil.writeLong((Id) this.subLabelIdOffHeap.zeroCopyReadFromByteBuf()),
+ this.sortValues(),
+ IdUtil.writeString((Id) this.otherVertexIdOffHeap.zeroCopyReadFromByteBuf()));
+ } else {
+ tmpCache = SplicingIdGenerator.concat(
+ IdUtil.writeString((Id) this.ownerVertexIdOffHeap.zeroCopyReadFromByteBuf()),
+ IdUtil.writeLong((Id) this.edgeLabelIdOffHeap.zeroCopyReadFromByteBuf()),
+ IdUtil.writeLong((Id) this.subLabelIdOffHeap.zeroCopyReadFromByteBuf()),
+ this.sortValues(),
+ IdUtil.writeString((Id) this.otherVertexIdOffHeap.zeroCopyReadFromByteBuf()));
+ }
+ byte[] tmpCacheBytes = tmpCache.getBytes(StandardCharsets.UTF_8);
+ this.cacheOffHeap = (ByteBuf) memoryPool.requireMemory(tmpCacheBytes.length, memoryPool);
+ this.cacheOffHeap.markReaderIndex();
+ this.cacheOffHeap.writeBytes(tmpCacheBytes);
+ return tmpCache;
+ }
+
+ @Override
+ public int hashCode() {
+ if (this.directed) {
+ return Objects.hash(this.ownerVertexIdOffHeap,
+ this.direction,
+ this.edgeLabelIdOffHeap,
+ this.subLabelIdOffHeap,
+ this.sortValuesOffHeap,
+ this.otherVertexIdOffHeap);
+ } else {
+ return Objects.hash(this.otherVertexIdOffHeap,
+ this.edgeLabelIdOffHeap,
+ this.subLabelIdOffHeap,
+ this.sortValuesOffHeap,
+ this.ownerVertexIdOffHeap);
+ }
+ }
+
+ @Override
+ public boolean equals(Object object) {
+ if (!(object instanceof EdgeIdOffHeap)) {
+ return false;
+ }
+ EdgeIdOffHeap other = (EdgeIdOffHeap) object;
+ if (this.directed) {
+ return this.ownerVertexIdOffHeap.equals(other.ownerVertexIdOffHeap) &&
+ this.direction == other.direction &&
+ this.edgeLabelIdOffHeap.equals(other.edgeLabelIdOffHeap) &&
+ this.subLabelIdOffHeap.equals(other.subLabelIdOffHeap) &&
+ this.sortValuesOffHeap.equals(other.sortValuesOffHeap) &&
+ this.otherVertexIdOffHeap.equals(other.otherVertexIdOffHeap);
+ } else {
+ return this.otherVertexIdOffHeap.equals(other.otherVertexIdOffHeap) &&
+ this.edgeLabelIdOffHeap.equals(other.edgeLabelIdOffHeap) &&
+ this.subLabelIdOffHeap.equals(other.subLabelIdOffHeap) &&
+ this.sortValuesOffHeap.equals(other.sortValuesOffHeap) &&
+ this.ownerVertexIdOffHeap.equals(other.ownerVertexIdOffHeap);
+ }
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/LongIdOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/LongIdOffHeap.java
new file mode 100644
index 0000000000..8d86f84a88
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/LongIdOffHeap.java
@@ -0,0 +1,148 @@
+/*
+ * 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.memory.consumer.impl.id;
+
+import static org.apache.hugegraph.backend.id.IdGenerator.compareType;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.id.IdGenerator;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.util.NumericUtil;
+
+import io.netty.buffer.ByteBuf;
+
+public class LongIdOffHeap extends IdGenerator.LongId implements OffHeapObject {
+
+ private ByteBuf idOffHeap;
+
+ public LongIdOffHeap(MemoryPool memoryPool, long id) {
+ super(id);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ public LongIdOffHeap(MemoryPool memoryPool, byte[] bytes) {
+ super(bytes);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ try {
+ return new IdGenerator.LongId(idOffHeap.readLong());
+ } finally {
+ idOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ this.idOffHeap = (ByteBuf) memoryPool.requireMemory(Long.BYTES, memoryPool);
+ this.idOffHeap.markReaderIndex();
+ this.idOffHeap.writeLong(id);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.id = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(idOffHeap);
+ }
+
+ @Override
+ public long asLong() {
+ try {
+ return idOffHeap.readLong();
+ } finally {
+ idOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public Object asObject() {
+ return this.asLong();
+ }
+
+ @Override
+ public String asString() {
+ return Long.toString(this.asLong());
+ }
+
+ @Override
+ public byte[] asBytes() {
+ return NumericUtil.longToBytes(this.asLong());
+ }
+
+ @Override
+ public int compareTo(Id other) {
+ int cmp = compareType(this, other);
+ if (cmp != 0) {
+ return cmp;
+ }
+ return Long.compare(this.asLong(), other.asLong());
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(idOffHeap);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (!(other instanceof Number)) {
+ return false;
+ }
+ return this.asLong() == ((Number) other).longValue();
+ }
+
+ @Override
+ public String toString() {
+ return String.valueOf(this.asLong());
+ }
+
+ @Override
+ public int intValue() {
+ return (int) this.asLong();
+ }
+
+ @Override
+ public long longValue() {
+ return this.asLong();
+ }
+
+ @Override
+ public float floatValue() {
+ return this.asLong();
+ }
+
+ @Override
+ public double doubleValue() {
+ return this.asLong();
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/ObjectIdOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/ObjectIdOffHeap.java
new file mode 100644
index 0000000000..d47d955a4d
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/ObjectIdOffHeap.java
@@ -0,0 +1,89 @@
+/*
+ * 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.memory.consumer.impl.id;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hugegraph.backend.id.IdGenerator;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.util.FurySerializationUtil;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+
+public class ObjectIdOffHeap extends IdGenerator.ObjectId implements OffHeapObject {
+
+ private ByteBuf objectOffHeap;
+
+ public ObjectIdOffHeap(MemoryPool memoryPool, Object object) {
+ super(object);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ return new IdGenerator.ObjectId(FurySerializationUtil.FURY.deserialize(
+ ByteBufUtil.getBytes(this.objectOffHeap)));
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ byte[] bytes = FurySerializationUtil.FURY.serialize(object);
+ this.objectOffHeap = (ByteBuf) memoryPool.requireMemory(bytes.length, memoryPool);
+ this.objectOffHeap.markReaderIndex();
+ this.objectOffHeap.writeBytes(bytes);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.object = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(objectOffHeap);
+ }
+
+ @Override
+ public Object asObject() {
+ return FurySerializationUtil.FURY.deserialize(ByteBufUtil.getBytes(objectOffHeap));
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(objectOffHeap);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (!(other instanceof ObjectIdOffHeap)) {
+ return false;
+ }
+ return this.objectOffHeap.equals(((ObjectIdOffHeap) other).objectOffHeap);
+ }
+
+ @Override
+ public String toString() {
+ return super.toString();
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/QueryIdOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/QueryIdOffHeap.java
new file mode 100644
index 0000000000..8a8dd4d1ee
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/QueryIdOffHeap.java
@@ -0,0 +1,114 @@
+/*
+ * 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.memory.consumer.impl.id;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hugegraph.backend.cache.CachedBackendStore;
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.query.Query;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+
+public class QueryIdOffHeap extends CachedBackendStore.QueryId implements OffHeapObject {
+
+ private ByteBuf queryOffHeap;
+
+ public QueryIdOffHeap(MemoryPool memoryPool, Query q) {
+ super(q);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ try {
+ return new CachedBackendStore.QueryId(
+ this.queryOffHeap.toString(StandardCharsets.UTF_8),
+ this.hashCode);
+ } finally {
+ queryOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ byte[] stringBytes = query.getBytes((StandardCharsets.UTF_8));
+ this.queryOffHeap = (ByteBuf) memoryPool.requireMemory(stringBytes.length, memoryPool);
+ this.queryOffHeap.markReaderIndex();
+ this.queryOffHeap.writeBytes(stringBytes);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.query = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(queryOffHeap);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (!(other instanceof QueryIdOffHeap)) {
+ return false;
+ }
+ return this.queryOffHeap.equals(((QueryIdOffHeap) other).queryOffHeap);
+ }
+
+ @Override
+ public int compareTo(Id o) {
+ return this.asString().compareTo(o.asString());
+ }
+
+ @Override
+ public Object asObject() {
+ return this.asString();
+ }
+
+ @Override
+ public String asString() {
+ try {
+ return this.queryOffHeap.toString(StandardCharsets.UTF_8);
+ } finally {
+ this.queryOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public byte[] asBytes() {
+ return ByteBufUtil.getBytes(this.queryOffHeap);
+ }
+
+ @Override
+ public String toString() {
+ return this.asString();
+ }
+
+ @Override
+ public int length() {
+ return this.asString().length();
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/StringIdOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/StringIdOffHeap.java
new file mode 100644
index 0000000000..be96c2c963
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/StringIdOffHeap.java
@@ -0,0 +1,140 @@
+/*
+ * 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.memory.consumer.impl.id;
+
+import static org.apache.hugegraph.backend.id.IdGenerator.compareType;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.id.IdGenerator;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.jetbrains.annotations.TestOnly;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+
+public class StringIdOffHeap extends IdGenerator.StringId implements OffHeapObject {
+
+ private ByteBuf idOffHeap;
+
+ public StringIdOffHeap(MemoryPool memoryPool, String id) {
+ super(id);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ public StringIdOffHeap(MemoryPool memoryPool, byte[] bytes) {
+ super(bytes);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ try {
+ return new IdGenerator.StringId(idOffHeap.toString(StandardCharsets.UTF_8));
+ } finally {
+ idOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ byte[] stringBytes = id.getBytes((StandardCharsets.UTF_8));
+ this.idOffHeap = (ByteBuf) memoryPool.requireMemory(stringBytes.length, memoryPool);
+ this.idOffHeap.markReaderIndex();
+ this.idOffHeap.writeBytes(stringBytes);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.id = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(idOffHeap);
+ }
+
+ @Override
+ public Object asObject() {
+ return this.asString();
+ }
+
+ @Override
+ public String asString() {
+ try {
+ return this.idOffHeap.toString(StandardCharsets.UTF_8);
+ } finally {
+ this.idOffHeap.resetReaderIndex();
+ }
+ }
+
+ @Override
+ public long asLong() {
+ return Long.parseLong(this.asString());
+ }
+
+ @Override
+ public byte[] asBytes() {
+ return ByteBufUtil.getBytes(this.idOffHeap);
+ }
+
+ @Override
+ public int length() {
+ return this.asString().length();
+ }
+
+ @Override
+ public int compareTo(Id other) {
+ int cmp = compareType(this, other);
+ if (cmp != 0) {
+ return cmp;
+ }
+ return this.asString().compareTo(other.asString());
+ }
+
+ @Override
+ public int hashCode() {
+ return this.idOffHeap.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (!(other instanceof StringIdOffHeap)) {
+ return false;
+ }
+ return this.idOffHeap.equals(((StringIdOffHeap) other).idOffHeap);
+ }
+
+ @Override
+ public String toString() {
+ return this.asString();
+ }
+
+ @TestOnly
+ public ByteBuf getIdOffHeap() {
+ return idOffHeap;
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/UuidIdOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/UuidIdOffHeap.java
new file mode 100644
index 0000000000..ed4b0399b5
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/id/UuidIdOffHeap.java
@@ -0,0 +1,142 @@
+/*
+ * 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.memory.consumer.impl.id;
+
+import static org.apache.hugegraph.backend.id.IdGenerator.compareType;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.id.IdGenerator;
+import org.apache.hugegraph.backend.serializer.BytesBuffer;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.util.FurySerializationUtil;
+import org.apache.hugegraph.memory.util.SerializationRuntimeException;
+import org.apache.hugegraph.util.E;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+
+public class UuidIdOffHeap extends IdGenerator.UuidId implements OffHeapObject {
+
+ private static final Logger LOG = LoggerFactory.getLogger(UuidIdOffHeap.class);
+ private ByteBuf idOffHeap;
+
+ public UuidIdOffHeap(MemoryPool memoryPool, String string) {
+ super(string);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ public UuidIdOffHeap(MemoryPool memoryPool, byte[] bytes) {
+ super(bytes);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ public UuidIdOffHeap(MemoryPool memoryPool, UUID uuid) {
+ super(uuid);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ return new IdGenerator.UuidId((UUID) FurySerializationUtil.FURY.deserialize(
+ ByteBufUtil.getBytes(this.idOffHeap)));
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ byte[] bytes = FurySerializationUtil.FURY.serialize(uuid);
+ this.idOffHeap = (ByteBuf) memoryPool.requireMemory(bytes.length, memoryPool);
+ this.idOffHeap.markReaderIndex();
+ this.idOffHeap.writeBytes(bytes);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.uuid = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(idOffHeap);
+ }
+
+ @Override
+ public Object asObject() {
+ return FurySerializationUtil.FURY.deserialize(
+ ByteBufUtil.getBytes(this.idOffHeap));
+ }
+
+ @Override
+ public String asString() {
+ return this.asObject().toString();
+ }
+
+ @Override
+ public byte[] asBytes() {
+ try (BytesBuffer buffer = BytesBuffer.allocate(16)) {
+ UUID tmp = (UUID) this.asObject();
+ buffer.writeLong(tmp.getMostSignificantBits());
+ buffer.writeLong(tmp.getLeastSignificantBits());
+ return buffer.bytes();
+ } catch (IOException e) {
+ LOG.error("Unexpected error occurs when allocate bytesBuffer.", e);
+ throw new SerializationRuntimeException(e);
+ }
+ }
+
+ @Override
+ public int compareTo(Id other) {
+ E.checkNotNull(other, "compare id");
+ int cmp = compareType(this, other);
+ if (cmp != 0) {
+ return cmp;
+ }
+ return ((UUID) this.asObject()).compareTo((UUID) other.asObject());
+ }
+
+ @Override
+ public int hashCode() {
+ return this.idOffHeap.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (!(other instanceof UuidIdOffHeap)) {
+ return false;
+ }
+ return this.asObject().equals(((UuidIdOffHeap) other).asObject());
+ }
+
+ @Override
+ public String toString() {
+ return this.asString();
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/property/HugeEdgePropertyOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/property/HugeEdgePropertyOffHeap.java
new file mode 100644
index 0000000000..d2f21a414c
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/property/HugeEdgePropertyOffHeap.java
@@ -0,0 +1,94 @@
+/*
+ * 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.memory.consumer.impl.property;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.util.FurySerializationUtil;
+import org.apache.hugegraph.schema.PropertyKey;
+import org.apache.hugegraph.structure.HugeEdge;
+import org.apache.hugegraph.structure.HugeEdgeProperty;
+import org.apache.hugegraph.structure.HugeElement;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+
+public class HugeEdgePropertyOffHeap extends HugeEdgeProperty implements OffHeapObject {
+
+ private ByteBuf valueOffHeap;
+
+ public HugeEdgePropertyOffHeap(MemoryPool memoryPool, HugeElement owner, PropertyKey key,
+ V value) {
+ super(owner, key, value);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ return new HugeEdgeProperty<>(this.owner, this.pkey,
+ FurySerializationUtil.FURY.deserialize(
+ ByteBufUtil.getBytes(this.valueOffHeap)));
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ byte[] bytes = FurySerializationUtil.FURY.serialize(this.value);
+ this.valueOffHeap = (ByteBuf) memoryPool.requireMemory(bytes.length, memoryPool);
+ this.valueOffHeap.markReaderIndex();
+ this.valueOffHeap.writeBytes(bytes);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.value = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(valueOffHeap);
+ }
+
+ @Override
+ public HugeEdgeProperty switchEdgeOwner() {
+ assert this.owner instanceof HugeEdge;
+ return new HugeEdgeProperty<>(((HugeEdge) this.owner).switchOwner(),
+ this.pkey, this.value());
+ }
+
+ @Override
+ public Object serialValue(boolean encodeNumber) {
+ return this.pkey.serialValue(this.value(), encodeNumber);
+ }
+
+ @Override
+ public boolean isPresent() {
+ return this.value() != null;
+ }
+
+ @Override
+ public V value() throws NoSuchElementException {
+ return (V) FurySerializationUtil.FURY.deserialize(
+ ByteBufUtil.getBytes(this.valueOffHeap));
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/property/HugeVertexPropertyOffHeap.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/property/HugeVertexPropertyOffHeap.java
new file mode 100644
index 0000000000..1beb759cf0
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/consumer/impl/property/HugeVertexPropertyOffHeap.java
@@ -0,0 +1,86 @@
+/*
+ * 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.memory.consumer.impl.property;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.util.FurySerializationUtil;
+import org.apache.hugegraph.schema.PropertyKey;
+import org.apache.hugegraph.structure.HugeElement;
+import org.apache.hugegraph.structure.HugeVertexProperty;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+
+public class HugeVertexPropertyOffHeap extends HugeVertexProperty implements OffHeapObject {
+
+ private ByteBuf valueOffHeap;
+
+ public HugeVertexPropertyOffHeap(MemoryPool memoryPool, HugeElement owner, PropertyKey key,
+ V value) {
+ super(owner, key, value);
+ serializeSelfToByteBuf(memoryPool);
+ releaseOriginalVarsOnHeap();
+ memoryPool.bindMemoryConsumer(this);
+ }
+
+ @Override
+ public Object zeroCopyReadFromByteBuf() {
+ return new HugeVertexProperty<>(this.owner, this.pkey,
+ FurySerializationUtil.FURY.deserialize(
+ ByteBufUtil.getBytes(this.valueOffHeap)));
+ }
+
+ @Override
+ public void serializeSelfToByteBuf(MemoryPool memoryPool) {
+ byte[] bytes = FurySerializationUtil.FURY.serialize(this.value);
+ this.valueOffHeap = (ByteBuf) memoryPool.requireMemory(bytes.length, memoryPool);
+ this.valueOffHeap.markReaderIndex();
+ this.valueOffHeap.writeBytes(bytes);
+ }
+
+ @Override
+ public void releaseOriginalVarsOnHeap() {
+ this.value = null;
+ }
+
+ @Override
+ public List getAllMemoryBlock() {
+ return Collections.singletonList(valueOffHeap);
+ }
+
+ @Override
+ public Object serialValue(boolean encodeNumber) {
+ return this.pkey.serialValue(this.value(), encodeNumber);
+ }
+
+ @Override
+ public boolean isPresent() {
+ return this.value() != null;
+ }
+
+ @Override
+ public V value() throws NoSuchElementException {
+ return (V) FurySerializationUtil.FURY.deserialize(
+ ByteBufUtil.getBytes(this.valueOffHeap));
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/AbstractMemoryPool.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/AbstractMemoryPool.java
new file mode 100644
index 0000000000..29b951c262
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/AbstractMemoryPool.java
@@ -0,0 +1,249 @@
+/*
+ * 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.memory.pool;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.impl.MemoryPoolStats;
+import org.jetbrains.annotations.TestOnly;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractMemoryPool implements MemoryPool {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AbstractMemoryPool.class);
+ protected final Queue children =
+ new PriorityQueue<>((o1, o2) -> (int) (o2.getFreeBytes() - o1.getFreeBytes()));
+ protected final MemoryManager memoryManager;
+ // Allocation, deAllocation, arbitration must be serial which is controlled by this lock.
+ protected final ReentrantLock memoryActionLock = new ReentrantLock();
+ protected final Condition condition = memoryActionLock.newCondition();
+ protected final AtomicBoolean isBeingArbitrated = new AtomicBoolean(false);
+ protected final MemoryPoolStats stats;
+ protected boolean isClosed = false;
+ private MemoryPool parent;
+
+ public AbstractMemoryPool(MemoryPool parent, String memoryPoolName,
+ MemoryPoolStats.MemoryPoolType type, MemoryManager memoryManager) {
+ this.parent = parent;
+ this.stats = new MemoryPoolStats(memoryPoolName, type);
+ this.memoryManager = memoryManager;
+ }
+
+ protected long tryToReclaimLocalMemoryWithoutLock(long neededBytes, MemoryPool requestingPool) {
+ long totalReclaimedBytes = 0;
+ try {
+ totalReclaimedBytes = reclaimChildren(neededBytes, requestingPool);
+ return totalReclaimedBytes;
+ } finally {
+ if (totalReclaimedBytes > 0) {
+ this.stats.setNumShrinks(this.stats.getNumShrinks() + 1);
+ }
+ this.stats.setAllocatedBytes(
+ this.stats.getAllocatedBytes() - totalReclaimedBytes);
+ this.isBeingArbitrated.set(false);
+ }
+ }
+
+ @Override
+ public long tryToReclaimLocalMemory(long neededBytes, MemoryPool requestingPool) {
+ this.memoryActionLock.lock();
+ long totalReclaimedBytes = 0;
+ try {
+ totalReclaimedBytes = reclaimChildren(neededBytes, requestingPool);
+ return totalReclaimedBytes;
+ } finally {
+ if (totalReclaimedBytes > 0) {
+ this.stats.setNumShrinks(this.stats.getNumShrinks() + 1);
+ }
+ this.stats.setAllocatedBytes(
+ this.stats.getAllocatedBytes() - totalReclaimedBytes);
+ this.isBeingArbitrated.set(false);
+ this.condition.signalAll();
+ this.memoryActionLock.unlock();
+ }
+ }
+
+ private long reclaimChildren(long neededBytes, MemoryPool requestingPool) {
+ LOG.debug("[{}] tryToReclaimLocalMemory: neededBytes={}", this, neededBytes);
+ this.isBeingArbitrated.set(true);
+ long totalReclaimedBytes = 0;
+ long currentNeededBytes = neededBytes;
+ for (MemoryPool child : this.children) {
+ if (child.equals(requestingPool)) {
+ continue;
+ }
+ long reclaimedMemory =
+ child.tryToReclaimLocalMemory(currentNeededBytes, requestingPool);
+ if (reclaimedMemory > 0) {
+ currentNeededBytes -= reclaimedMemory;
+ totalReclaimedBytes += reclaimedMemory;
+ // Reclaim enough memory.
+ if (currentNeededBytes <= 0) {
+ break;
+ }
+ }
+ }
+ LOG.info("[{}] has finished to reclaim memory: totalReclaimedBytes={}, " +
+ "neededBytes={}, snapshot-[{}]",
+ this,
+ totalReclaimedBytes, neededBytes, this.getSnapShot());
+ return totalReclaimedBytes;
+ }
+
+ /**
+ * called when one layer pool is successfully executed and exited.
+ */
+ @Override
+ public void releaseSelf(String reason, boolean isTriggeredByOOM) {
+ try {
+ if (!isTriggeredByOOM) {
+ this.memoryActionLock.lock();
+ if (this.isBeingArbitrated.get()) {
+ this.condition.await();
+ }
+ }
+ LOG.debug("[{}] starts to releaseSelf", this);
+ this.isClosed = true;
+ // gc self from father
+ Optional.ofNullable(this.parent).ifPresent(parent -> parent.gcChildPool(this, false,
+ isTriggeredByOOM));
+ // gc all children
+ Set copiedChildren = new HashSet<>(this.children);
+ // since `gcChildPool` will remove elements from this.children, we need to traverse an
+ // immutable copy of this.children.
+ for (MemoryPool child : copiedChildren) {
+ gcChildPool(child, true, isTriggeredByOOM);
+ }
+ copiedChildren.clear();
+ LOG.info("[{}] finishes to releaseSelf because of {}", this, reason);
+ } catch (InterruptedException e) {
+ LOG.error("Failed to release self because ", e);
+ Thread.currentThread().interrupt();
+ } finally {
+ if (!isTriggeredByOOM) {
+ this.memoryActionLock.unlock();
+ }
+ // Make these objs be GCed by JVM quickly.
+ this.parent = null;
+ this.children.clear();
+ }
+ }
+
+ @Override
+ public void gcChildPool(MemoryPool child, boolean force, boolean isTriggeredInternal) {
+ if (force) {
+ child.releaseSelf(String.format("[%s] releaseChildPool", this), isTriggeredInternal);
+ return;
+ }
+ // reclaim child's memory and update stats
+ this.stats.setAllocatedBytes(
+ stats.getAllocatedBytes() - child.getAllocatedBytes());
+ this.stats.setUsedBytes(this.stats.getUsedBytes() - child.getUsedBytes());
+ this.children.remove(child);
+ }
+
+ @Override
+ public Object tryToAcquireMemoryInternal(long bytes) {
+ if (isClosed) {
+ LOG.warn("[{}] is already closed, will abort this allocate", this);
+ return 0;
+ }
+ // just record how much memory is used(update stats)
+ this.stats.setUsedBytes(this.stats.getUsedBytes() + bytes);
+ this.stats.setCumulativeBytes(this.stats.getCumulativeBytes() + bytes);
+ return null;
+ }
+
+ @Override
+ public void bindMemoryConsumer(OffHeapObject offHeapObject) {
+ // default do nothing
+ }
+
+ @Override
+ public Object requireMemory(long bytes, MemoryPool requestingPool) {
+ return null;
+ }
+
+ @Override
+ public long getMaxCapacityBytes() {
+ return Optional.of(this.stats).map(MemoryPoolStats::getMaxCapacity).orElse(0L);
+ }
+
+ @Override
+ public long getUsedBytes() {
+ return Optional.of(this.stats).map(MemoryPoolStats::getUsedBytes).orElse(0L);
+ }
+
+ @Override
+ public long getFreeBytes() {
+ return Optional.of(this.stats)
+ .map(stats -> stats.getAllocatedBytes() - stats.getUsedBytes()).orElse(0L);
+ }
+
+ @Override
+ public long getAllocatedBytes() {
+ return Optional.of(this.stats).map(MemoryPoolStats::getAllocatedBytes).orElse(0L);
+ }
+
+ @Override
+ public MemoryPoolStats getSnapShot() {
+ return this.stats;
+ }
+
+ @Override
+ public MemoryPool getParentPool() {
+ return this.parent;
+ }
+
+ @Override
+ public String toString() {
+ return this.getName();
+ }
+
+ public String getName() {
+ return this.stats.getMemoryPoolName();
+ }
+
+ @Override
+ public MemoryPool findRootQueryPool() {
+ if (this.parent == null) {
+ return this;
+ }
+ return getParentPool().findRootQueryPool();
+ }
+
+ @Override
+ public void setMaxCapacityBytes(long maxCapacityBytes) {
+ this.stats.setMaxCapacity(maxCapacityBytes);
+ }
+
+ @TestOnly
+ public int getChildrenCount() {
+ return this.children.size();
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/MemoryPool.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/MemoryPool.java
new file mode 100644
index 0000000000..74c364febb
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/MemoryPool.java
@@ -0,0 +1,107 @@
+/*
+ * 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.memory.pool;
+
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.impl.MemoryPoolStats;
+import org.apache.hugegraph.memory.util.OutOfMemoryException;
+import org.jetbrains.annotations.TestOnly;
+
+public interface MemoryPool {
+
+ MemoryPoolStats getSnapShot();
+
+ /**
+ * Try to reclaim self's free memory, triggered by an arbitration.
+ *
+ * @param neededBytes reclaim goal, which means this reclaim will try to collect neededBytes.
+ * @param requestingPool the memoryPool that requests this arbitration.
+ * @return reclaimed memory in bytes.
+ */
+ long tryToReclaimLocalMemory(long neededBytes, MemoryPool requestingPool);
+
+ /**
+ * Require to be allocated with a memory block. This method contains two steps:
+ *
+ * 1. Request manager for a logic allocation. Throw OOM exception if memory exhausted.
+ * 2. Request allocator for a real allocation. Return real memory block.
+ *
+ * @param bytes needed bytes.
+ * @param requestingPool the memoryPool that requests this allocation.
+ * @return memory block. ByteBuf if using off-heap, byte[] if using on-heap.
+ */
+ Object requireMemory(long bytes, MemoryPool requestingPool);
+
+ /**
+ * Request MemoryManager for a logic allocation. This method is used internal in {@code
+ * requireMemory}.
+ *
+ * @param bytes needed bytes.
+ * @param requestingPool the memoryPool that requests this allocation.
+ * @return the memory size that upper manager can provide.
+ * @throws OutOfMemoryException if there isn't enough memory in manager.
+ */
+ long requestMemoryInternal(long bytes, MemoryPool requestingPool) throws OutOfMemoryException;
+
+ /**
+ * Request allocator for a real allocation. This method is used internal in {@code
+ * requireMemory}.
+ *
+ * @param bytes needed bytes.
+ * @return memory block. ByteBuf if using off-heap, byte[] if using on-heap.
+ */
+ Object tryToAcquireMemoryInternal(long bytes);
+
+ /**
+ * Release all self's resources. Called by user or called automatically by itself when OOM.
+ *
+ * @param reason release reason, for logging.
+ * @param isTriggeredByOOM if true, it is called when OOM. if false, called by user.
+ */
+ void releaseSelf(String reason, boolean isTriggeredByOOM);
+
+ /**
+ * Called by `releaseSelf` to release children's resource.
+ *
+ * @param child: child pool
+ * @param force: if false, called to gc self from father
+ * @param isTriggeredInternal: passed from upper caller `releaseSelf`
+ */
+ void gcChildPool(MemoryPool child, boolean force, boolean isTriggeredInternal);
+
+ long getAllocatedBytes();
+
+ long getUsedBytes();
+
+ long getFreeBytes();
+
+ long getMaxCapacityBytes();
+
+ MemoryPool getParentPool();
+
+ MemoryPool findRootQueryPool();
+
+ MemoryPool addChildPool(String name);
+
+ void bindMemoryConsumer(OffHeapObject offHeapObject);
+
+ void setMaxCapacityBytes(long maxCapacityBytes);
+
+ @TestOnly
+ int getChildrenCount();
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/MemoryPoolStats.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/MemoryPoolStats.java
new file mode 100644
index 0000000000..0409a05a65
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/MemoryPoolStats.java
@@ -0,0 +1,144 @@
+/*
+ * 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.memory.pool.impl;
+
+public class MemoryPoolStats {
+
+ private final String memoryPoolName;
+ private final MemoryPoolType memoryPoolType;
+ private long maxCapacity;
+ private long usedBytes;
+ // it represents the cumulative used bytes.
+ private long cumulativeBytes;
+ private long allocatedBytes;
+
+ // it represents the shrinking num of allocatedBytes
+ private long numShrinks;
+ // for query pool, it represents the enlarging num of maxCapacity; for other pools, it
+ // represents the enlarging num of allocatedBytes
+ private long numExpands;
+ private long numAborts;
+
+ public MemoryPoolStats(String MemoryPoolName, MemoryPoolType memoryPoolType) {
+ this.memoryPoolName = MemoryPoolName;
+ this.memoryPoolType = memoryPoolType;
+ }
+
+ @Override
+ public String toString() {
+ switch (memoryPoolType) {
+ case TASK:
+ return String.format("%s: {usedBytes[%d], cumulativeBytes[%d], " +
+ "allocatedBytes[%d], numShrinks[%d], numAborts[%d]}.",
+ memoryPoolName, usedBytes, cumulativeBytes, allocatedBytes,
+ numShrinks, numAborts);
+ case OPERATOR:
+ return String.format("%s: {usedBytes[%d], cumulativeBytes[%d], " +
+ "allocatedBytes[%d], numShrinks[%d], numExpands[%d], " +
+ "numAborts[%d]}.",
+ memoryPoolName, usedBytes, cumulativeBytes, allocatedBytes,
+ numShrinks, numExpands, numAborts);
+ case QUERY:
+ default:
+ return String.format("%s: {maxCapacity[%d], usedBytes[%d]," +
+ "cumulativeBytes[%d], allocatedBytes[%d], numShrinks[%d], " +
+ "numExpands[%d], numAborts[%d]}.", memoryPoolName, maxCapacity,
+ usedBytes,
+ cumulativeBytes, allocatedBytes, numShrinks, numExpands,
+ numAborts);
+ }
+
+ }
+
+ public String getMemoryPoolName() {
+ return this.memoryPoolName;
+ }
+
+ public long getUsedBytes() {
+ return this.usedBytes;
+ }
+
+ public void setUsedBytes(long usedBytes) {
+ this.usedBytes = usedBytes;
+ }
+
+ public long getCumulativeBytes() {
+ return this.cumulativeBytes;
+ }
+
+ public void setCumulativeBytes(long cumulativeBytes) {
+ this.cumulativeBytes = cumulativeBytes;
+ }
+
+ public long getAllocatedBytes() {
+ return this.allocatedBytes;
+ }
+
+ public void setAllocatedBytes(long allocatedBytes) {
+ this.allocatedBytes = allocatedBytes;
+ }
+
+ public long getNumShrinks() {
+ return this.numShrinks;
+ }
+
+ public void setNumShrinks(long numShrinks) {
+ this.numShrinks = numShrinks;
+ }
+
+ public long getNumExpands() {
+ return this.numExpands;
+ }
+
+ public void setNumExpands(long numExpands) {
+ this.numExpands = numExpands;
+ }
+
+ public long getNumAborts() {
+ return this.numAborts;
+ }
+
+ public void setNumAborts(long numAborts) {
+ this.numAborts = numAborts;
+ }
+
+ public long getMaxCapacity() {
+ return this.maxCapacity;
+ }
+
+ public void setMaxCapacity(long maxCapacity) {
+ this.maxCapacity = maxCapacity;
+ }
+
+ public enum MemoryPoolType {
+ QUERY(0),
+ TASK(1),
+ OPERATOR(2),
+ UNKNOWN(3);
+
+ private final int code;
+
+ MemoryPoolType(int code) {
+ this.code = code;
+ }
+
+ public int getCode() {
+ return this.code;
+ }
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/OperatorMemoryPool.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/OperatorMemoryPool.java
new file mode 100644
index 0000000000..af9121ee46
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/OperatorMemoryPool.java
@@ -0,0 +1,231 @@
+/*
+ * 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.memory.pool.impl;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.allocator.MemoryAllocator;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.pool.AbstractMemoryPool;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.util.OutOfMemoryException;
+import org.apache.hugegraph.memory.util.RoundUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OperatorMemoryPool extends AbstractMemoryPool {
+
+ private static final Logger LOG = LoggerFactory.getLogger(OperatorMemoryPool.class);
+ private final MemoryAllocator memoryAllocator;
+ private final Set offHeapObjects;
+
+ public OperatorMemoryPool(MemoryPool parent, String poolName,
+ MemoryAllocator memoryAllocator, MemoryManager memoryManager) {
+ super(parent, poolName, MemoryPoolStats.MemoryPoolType.OPERATOR, memoryManager);
+ this.memoryAllocator = memoryAllocator;
+ this.offHeapObjects = new HashSet<>();
+ }
+
+ @Override
+ public MemoryPool addChildPool(String name) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void bindMemoryConsumer(OffHeapObject offHeapObject) {
+ this.offHeapObjects.add(offHeapObject);
+ }
+
+ @Override
+ public void releaseSelf(String reason, boolean isTriggeredByOOM) {
+ super.releaseSelf(reason, isTriggeredByOOM);
+ // since it is already closed, its stats will not be updated. so here we can use its
+ // stats out of memoryActionLock.
+ this.memoryAllocator.returnMemoryToManager(getUsedBytes());
+ this.memoryManager.returnReclaimedTaskMemory(getAllocatedBytes());
+ // release memory consumer, release byte buffer.
+ this.offHeapObjects.forEach(memoryConsumer -> {
+ memoryConsumer.getAllMemoryBlock().forEach(memoryAllocator::releaseMemoryBlock);
+ });
+ this.offHeapObjects.clear();
+ this.resetStats();
+ }
+
+ @Override
+ public long tryToReclaimLocalMemory(long neededBytes, MemoryPool requestingPool) {
+ if (this.isClosed) {
+ LOG.warn("[{}] is already closed, will abort this reclaim", this);
+ return 0;
+ }
+ long reclaimableBytes = 0;
+ try {
+ if (!this.equals(requestingPool)) {
+ this.memoryActionLock.lock();
+ }
+ LOG.debug("[{}] tryToReclaimLocalMemory: neededBytes={}", this, neededBytes);
+ this.isBeingArbitrated.set(true);
+ // 1. try to reclaim self free memory
+ reclaimableBytes = getFreeBytes();
+ // try its best to reclaim memory
+ if (reclaimableBytes <= neededBytes) {
+ // 2. update stats
+ stats.setAllocatedBytes(stats.getUsedBytes());
+ LOG.info("[{}] has tried its best to reclaim memory: " +
+ "reclaimedBytes={}," +
+ " " +
+ "neededBytes={}, snapshot-[{}]",
+ this,
+ reclaimableBytes, neededBytes, this.getSnapShot());
+ return reclaimableBytes;
+ }
+ stats.setAllocatedBytes(stats.getAllocatedBytes() - neededBytes);
+ LOG.info("[{}] has reclaim enough memory: " +
+ "reclaimedBytes={}," +
+ " " +
+ "neededBytes={}, snapshot-[{}]",
+ this,
+ neededBytes, neededBytes, this.getSnapShot());
+ return neededBytes;
+ } finally {
+ if (reclaimableBytes > 0) {
+ this.stats.setNumShrinks(this.stats.getNumShrinks() + 1);
+ }
+ this.isBeingArbitrated.set(false);
+ if (!this.equals(requestingPool)) {
+ this.condition.signalAll();
+ this.memoryActionLock.unlock();
+ }
+ }
+ }
+
+ /**
+ * called by user
+ */
+ @Override
+ public Object requireMemory(long bytes, MemoryPool requestingPool) {
+ try {
+ // use lock to ensure the atomicity of the two-step operation
+ this.memoryActionLock.lock();
+ // if free memory is enough, use free memory directly.
+ if (getFreeBytes() >= bytes) {
+ LOG.debug("[{}] require {} bytes, there is enough free memory {} bytes, will " +
+ "require memory directly from self's free memory.",
+ this,
+ getFreeBytes(),
+ bytes);
+ } else {
+ // if free memory is not enough, try to request delta
+ long delta = bytes - getFreeBytes();
+ long ignoredRealAllocatedBytes = requestMemoryInternal(delta, requestingPool);
+ }
+ return tryToAcquireMemoryInternal(bytes);
+ } catch (OutOfMemoryException e) {
+ // Abort this query
+ LOG.warn("[{}] detected an OOM exception when request memory, will ABORT this " +
+ "query and release corresponding memory... snapshot-[{}]",
+ this, this.getSnapShot());
+ findRootQueryPool().releaseSelf(String.format(e.getMessage()), true);
+ return null;
+ } finally {
+ this.memoryActionLock.unlock();
+ }
+ }
+
+ /**
+ * This method will update `used` and `cumulative` stats.
+ */
+ @Override
+ public Object tryToAcquireMemoryInternal(long size) {
+ if (this.isClosed) {
+ LOG.warn("[{}] is already closed, will abort this allocate", this);
+ return null;
+ }
+ LOG.debug("[{}] tryToAcquireMemory: size={}", this, size);
+ // 1. update statistic
+ super.tryToAcquireMemoryInternal(size);
+ // 2. call parent to update statistic
+ getParentPool().tryToAcquireMemoryInternal(size);
+ // 3. allocate memory, currently use off-heap mode.
+ return this.memoryAllocator.tryToAllocate(size);
+ }
+
+ /**
+ * Operator need `size` bytes, operator pool will try to reserve some memory for it.
+ * This method will update `allocated` and `expand` stats.
+ */
+ @Override
+ public long requestMemoryInternal(long size, MemoryPool requestingPool) throws
+ OutOfMemoryException {
+ if (this.isClosed) {
+ LOG.warn("[{}] is already closed, will abort this request", this);
+ return 0;
+ }
+ try {
+ if (this.isBeingArbitrated.get()) {
+ this.condition.await();
+ }
+ LOG.debug("[{}] requestMemory: request size={}", this, size);
+ // 1. align size
+ long alignedSize = RoundUtil.sizeAlign(size);
+ // 2. reserve(round)
+ long neededMemorySize = calculateReserveMemoryDelta(alignedSize);
+ // 3. call father
+ long fatherRes =
+ getParentPool().requestMemoryInternal(neededMemorySize, requestingPool);
+ if (fatherRes <= 0) {
+ LOG.error("[{}] requestMemory failed because of OOM, request size={}", this,
+ size);
+ // if parentRes <= 0, indicating we don't get enough memory bytes.
+ // But we still need to allocate these memory bytes to operatorPool to ensure
+ // memory is conserved when parentRes < 0
+ if (fatherRes < 0) {
+ this.stats.setAllocatedBytes(this.stats.getAllocatedBytes() - fatherRes);
+ this.stats.setNumExpands(this.stats.getNumExpands() + 1);
+ }
+ this.stats.setNumAborts(this.stats.getNumAborts() + 1);
+ throw new OutOfMemoryException(String.format("%s requestMemory failed " +
+ "because of OOM, request " +
+ "size=%s", this, size));
+ }
+ // 4. update stats
+ this.stats.setAllocatedBytes(this.stats.getAllocatedBytes() + fatherRes);
+ this.stats.setNumExpands(this.stats.getNumExpands() + 1);
+ LOG.debug("[{}] requestMemory success: requestedMemorySize={}", this, fatherRes);
+ return fatherRes;
+ } catch (InterruptedException e) {
+ LOG.error("Failed to release self because ", e);
+ Thread.currentThread().interrupt();
+ return 0;
+ }
+ }
+
+ private long calculateReserveMemoryDelta(long size) {
+ return RoundUtil.roundDelta(getAllocatedBytes(), size);
+ }
+
+ private void resetStats() {
+ this.stats.setNumAborts(0);
+ this.stats.setNumExpands(0);
+ this.stats.setNumShrinks(0);
+ this.stats.setAllocatedBytes(0);
+ this.stats.setUsedBytes(0);
+ this.stats.setCumulativeBytes(0);
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/QueryMemoryPool.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/QueryMemoryPool.java
new file mode 100644
index 0000000000..4d6e492461
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/QueryMemoryPool.java
@@ -0,0 +1,163 @@
+/*
+ * 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.memory.pool.impl;
+
+import static org.apache.hugegraph.memory.MemoryManager.DELIMINATOR;
+import static org.apache.hugegraph.memory.MemoryManager.MAX_MEMORY_CAPACITY_FOR_ONE_QUERY;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.pool.AbstractMemoryPool;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.util.RoundUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class QueryMemoryPool extends AbstractMemoryPool {
+
+ private static final Logger LOG = LoggerFactory.getLogger(QueryMemoryPool.class);
+ private static final String TASK_MEMORY_POOL_NAME_PREFIX = "TaskMemoryPool";
+ private static final String EXPAND_SELF = "expand self's max capacity";
+ private static final String REQUEST_MEMORY = "request to allocate memory";
+
+ public QueryMemoryPool(String poolName, MemoryManager memoryManager) {
+ super(null, poolName, MemoryPoolStats.MemoryPoolType.QUERY, memoryManager);
+ this.stats.setMaxCapacity(MAX_MEMORY_CAPACITY_FOR_ONE_QUERY);
+ }
+
+ @Override
+ public long tryToReclaimLocalMemory(long neededBytes, MemoryPool requestingPool) {
+ if (isClosed) {
+ LOG.warn("[{}] is already closed, will abort this reclaim", this);
+ return 0;
+ }
+ if (this.equals(requestingPool.findRootQueryPool())) {
+ return super.tryToReclaimLocalMemoryWithoutLock(neededBytes, requestingPool);
+ }
+ return super.tryToReclaimLocalMemory(neededBytes, requestingPool);
+ }
+
+ @Override
+ public MemoryPool addChildPool(String name) {
+ int count = this.children.size();
+ String poolName =
+ TASK_MEMORY_POOL_NAME_PREFIX + DELIMINATOR + name + DELIMINATOR + count +
+ DELIMINATOR + System.currentTimeMillis();
+ MemoryPool taskMemoryPool = new TaskMemoryPool(this, poolName, this.memoryManager);
+ this.children.add(taskMemoryPool);
+ LOG.info("QueryPool-{} added task memory pool {}", this, taskMemoryPool);
+ return taskMemoryPool;
+ }
+
+ @Override
+ public long requestMemoryInternal(long bytes, MemoryPool requestingPool) {
+ if (this.isClosed) {
+ LOG.warn("[{}] is already closed, will abort this request", this);
+ return 0;
+ }
+ long requestedMemoryFromManager = 0;
+ long requestedMemoryFromArbitration = 0;
+ memoryActionLock.lock();
+ try {
+ if (this.isBeingArbitrated.get()) {
+ this.condition.await();
+ }
+ // 1. check whether self capacity is enough
+ if (getMaxCapacityBytes() - stats.getAllocatedBytes() < bytes) {
+ // 2.1 if not, first try to acquire memory from manager
+ long neededDelta = bytes - (getMaxCapacityBytes() - stats.getAllocatedBytes());
+ long managerReturnedMemoryInBytes = tryToExpandSelfCapacity(neededDelta);
+ if (managerReturnedMemoryInBytes > 0) {
+ this.stats.setMaxCapacity(getMaxCapacityBytes() + managerReturnedMemoryInBytes);
+ this.stats.setAllocatedBytes(this.stats.getAllocatedBytes() + bytes);
+ this.stats.setNumExpands(this.stats.getNumExpands() + 1);
+ requestedMemoryFromManager = bytes;
+ } else {
+ // 2.2 if requiring memory from manager failed, call manager to invoke arbitrate
+ requestedMemoryFromArbitration =
+ requestMemoryThroughArbitration(bytes, requestingPool);
+ }
+ } else {
+ // 3. if capacity is enough, check whether manager has enough memory.
+ if (this.memoryManager.handleRequestFromQueryPool(bytes, REQUEST_MEMORY) < 0) {
+ // 3.1 if memory manager doesn't have enough memory, call manager to invoke
+ // arbitrate
+ requestedMemoryFromArbitration =
+ requestMemoryThroughArbitration(bytes, requestingPool);
+ } else {
+ // 3.2 if memory manager has enough memory, return success
+ this.stats.setAllocatedBytes(this.stats.getAllocatedBytes() + bytes);
+ requestedMemoryFromManager = bytes;
+ }
+ }
+ if (requestedMemoryFromManager > 0) {
+ this.memoryManager.consumeAvailableMemory(requestedMemoryFromManager);
+ }
+ return requestedMemoryFromManager == 0 ? requestedMemoryFromArbitration :
+ requestedMemoryFromManager;
+ } catch (InterruptedException e) {
+ LOG.error("[{}] Failed to request memory because ", this, e);
+ Thread.currentThread().interrupt();
+ return 0;
+ } finally {
+ memoryActionLock.unlock();
+ }
+ }
+
+ private long tryToExpandSelfCapacity(long size) {
+ LOG.debug("[{}] try to expand its capacity: size={}", this, size);
+ long alignedSize = RoundUtil.sizeAlign(size);
+ long realNeededSize =
+ RoundUtil.roundDelta(getAllocatedBytes(), alignedSize);
+ return this.memoryManager.handleRequestFromQueryPool(realNeededSize, EXPAND_SELF);
+ }
+
+ private long requestMemoryThroughArbitration(long bytes, MemoryPool requestingPool) {
+ LOG.info("[{}] try to request memory from manager through arbitration: size={}, " +
+ "snapshot-[{}]",
+ this,
+ bytes,
+ this.getSnapShot());
+ long reclaimedBytes =
+ this.memoryManager.triggerLocalArbitration(this, bytes, requestingPool);
+ // 1. if arbitrate successes, update stats and return success
+ if (reclaimedBytes - bytes >= 0) {
+ // here we don't update capacity & reserved & allocated, because memory is
+ // reclaimed from queryPool itself.
+ return reclaimedBytes;
+ } else {
+ // 2. if still not enough, try to reclaim globally
+ long globalArbitrationNeededBytes = bytes - reclaimedBytes;
+ long globalReclaimedBytes = this.memoryManager.triggerGlobalArbitration(this,
+ globalArbitrationNeededBytes);
+ reclaimedBytes += globalReclaimedBytes;
+ // add capacity whether arbitration failed or not.
+ // NOTE: only bytes gained from global arbitration can be added to stats.
+ if (globalReclaimedBytes != 0) {
+ this.stats.setMaxCapacity(this.stats.getMaxCapacity() + globalReclaimedBytes);
+ this.stats.setAllocatedBytes(this.stats.getAllocatedBytes() + globalReclaimedBytes);
+ this.stats.setNumExpands(this.stats.getNumExpands() + 1);
+ }
+ // 3. if memory is enough, update stats and return success
+ if (reclaimedBytes - bytes >= 0) {
+ return reclaimedBytes;
+ }
+ }
+ // 4. if arbitrate fails, return -reclaimedBytes, indicating that request failed.
+ return -reclaimedBytes;
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/TaskMemoryPool.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/TaskMemoryPool.java
new file mode 100644
index 0000000000..813f2820bc
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/pool/impl/TaskMemoryPool.java
@@ -0,0 +1,119 @@
+/*
+ * 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.memory.pool.impl;
+
+import static org.apache.hugegraph.memory.MemoryManager.DELIMINATOR;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.allocator.NettyMemoryAllocator;
+import org.apache.hugegraph.memory.pool.AbstractMemoryPool;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.util.OutOfMemoryException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TaskMemoryPool extends AbstractMemoryPool {
+
+ private static final Logger LOG = LoggerFactory.getLogger(TaskMemoryPool.class);
+ private static final String OPERATOR_MEMORY_POOL_NAME_PREFIX = "OperatorMemoryPool";
+ // One thread corresponds to one task memory pool. Since the execution flow within a single
+ // thread is serial, there is only one working Operator pool in the thread at each time.
+ // This variable helps the execution flow obtain the memory management context.
+ private MemoryPool CURRENT_WORKING_OPERATOR_MEMORY_POOL = null;
+
+ public TaskMemoryPool(MemoryPool parent, String poolName, MemoryManager memoryManager) {
+ super(parent, poolName, MemoryPoolStats.MemoryPoolType.TASK, memoryManager);
+ }
+
+ public MemoryPool getCurrentWorkingOperatorMemoryPool() {
+ return CURRENT_WORKING_OPERATOR_MEMORY_POOL;
+ }
+
+ @Override
+ public long tryToReclaimLocalMemory(long neededBytes, MemoryPool requestingPool) {
+ if (isClosed) {
+ LOG.warn("[{}] is already closed, will abort this reclaim", this);
+ return 0;
+ }
+ if (this.findRootQueryPool().equals(requestingPool.findRootQueryPool())) {
+ return super.tryToReclaimLocalMemoryWithoutLock(neededBytes, requestingPool);
+ }
+ return super.tryToReclaimLocalMemory(neededBytes, requestingPool);
+ }
+
+ @Override
+ public void releaseSelf(String reason, boolean isTriggeredByOOM) {
+ super.releaseSelf(reason, isTriggeredByOOM);
+ this.memoryManager.removeCorrespondingTaskMemoryPool(Thread.currentThread().getName());
+ }
+
+ @Override
+ public MemoryPool addChildPool(String name) {
+ int count = this.children.size();
+ String poolName =
+ OPERATOR_MEMORY_POOL_NAME_PREFIX + DELIMINATOR + name + DELIMINATOR + count +
+ DELIMINATOR + System.currentTimeMillis();
+ MemoryPool operatorPool =
+ new OperatorMemoryPool(this, poolName,
+ new NettyMemoryAllocator(this.memoryManager),
+ this.memoryManager);
+ this.children.add(operatorPool);
+ CURRENT_WORKING_OPERATOR_MEMORY_POOL = operatorPool;
+ LOG.info("TaskPool-{} added operator memory pool {}", this, operatorPool);
+ return operatorPool;
+ }
+
+ @Override
+ public Object tryToAcquireMemoryInternal(long bytes) {
+ getParentPool().tryToAcquireMemoryInternal(bytes);
+ return super.tryToAcquireMemoryInternal(bytes);
+ }
+
+ @Override
+ public long requestMemoryInternal(long bytes, MemoryPool requestingPool) throws
+ OutOfMemoryException {
+ if (this.isClosed) {
+ LOG.warn("[{}] is already closed, will abort this request", this);
+ return 0;
+ }
+ memoryActionLock.lock();
+ try {
+ if (this.isBeingArbitrated.get()) {
+ this.condition.await();
+ }
+ long parentRes = getParentPool().requestMemoryInternal(bytes, requestingPool);
+ if (parentRes > 0) {
+ this.stats.setAllocatedBytes(this.stats.getAllocatedBytes() + parentRes);
+ this.stats.setNumExpands(this.stats.getNumExpands() + 1);
+ } else if (parentRes < 0){
+ // if parentRes < 0, indicating we don't get enough memory bytes. But we still
+ // need to allocate these memory bytes to operatorPool to ensure memory is
+ // conserved.
+ this.stats.setAllocatedBytes(this.stats.getAllocatedBytes() - parentRes);
+ this.stats.setNumExpands(this.stats.getNumExpands() + 1);
+ }
+ return parentRes;
+ } catch (InterruptedException e) {
+ LOG.error("Failed to release self because ", e);
+ Thread.currentThread().interrupt();
+ return 0;
+ } finally {
+ memoryActionLock.unlock();
+ }
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/FurySerializationUtil.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/FurySerializationUtil.java
new file mode 100644
index 0000000000..ff56caad7c
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/FurySerializationUtil.java
@@ -0,0 +1,35 @@
+/*
+ * 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.memory.util;
+
+import java.util.UUID;
+
+import org.apache.fury.Fury;
+import org.apache.fury.config.Language;
+
+public class FurySerializationUtil {
+
+ public static final Fury FURY = Fury.builder().withLanguage(Language.JAVA)
+ // not mandatory to register all class
+ .requireClassRegistration(false)
+ .build();
+
+ static {
+ FURY.register(UUID.class);
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/OffHeapMemoryUtil.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/OffHeapMemoryUtil.java
new file mode 100644
index 0000000000..a63ddc91b7
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/OffHeapMemoryUtil.java
@@ -0,0 +1,95 @@
+/*
+ * 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.memory.util;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import sun.misc.Unsafe;
+
+// NOTE: due to different layout of klass in various versions of JDK, this class may easily crash!
+public class OffHeapMemoryUtil {
+
+ private static final Logger LOG = LoggerFactory.getLogger(OffHeapMemoryUtil.class);
+ private static final Unsafe UNSAFE;
+
+ static {
+ Unsafe unsafe;
+ try {
+ Field unsafeField = Unsafe.class.getDeclaredField("theUnsafe");
+ unsafeField.setAccessible(true);
+ unsafe = (Unsafe) unsafeField.get(null);
+ } catch (Throwable cause) {
+ LOG.warn("sun.misc.Unsafe.theUnsafe: unavailable.", cause);
+ throw new UnsupportedOperationException("Unsafe is not supported in this platform.");
+ }
+ UNSAFE = unsafe;
+ }
+
+ public static long allocateObjectToOffHeap(Object obj) {
+ long size = sizeOf(obj);
+ System.out.println(size);
+ long address = getUnsafe().allocateMemory(size);
+ // test object was copied to off-heap
+ getUnsafe().copyMemory(
+ obj, // source object
+ 0, // source offset is zero - copy an entire object
+ null,
+ // destination is specified by absolute address, so destination object is null
+ address, // destination address
+ size
+ );
+ return address;
+ }
+
+ public static T retrieveObjectFromOffHeap(Class clazz, long address) throws
+ NoSuchFieldException,
+ NoSuchMethodException,
+ InstantiationException,
+ IllegalAccessException,
+ InvocationTargetException {
+ T pointer = clazz.getDeclaredConstructor().newInstance();
+ // pointer is just a handler that stores address of some object
+ // NOTE: we agree that all pointer classes must have a pointer field.
+ long offset = getUnsafe().objectFieldOffset(clazz.getDeclaredField("pointer"));
+ // set pointer to off-heap copy of the test object
+ getUnsafe().putLong(pointer, offset, address);
+ return pointer;
+ }
+
+ public static void freeOffHeapMemory(long address) {
+ getUnsafe().freeMemory(address);
+ }
+
+ private static Unsafe getUnsafe() {
+ return UNSAFE;
+ }
+
+ private static long sizeOf(Object object) {
+ Unsafe unsafe = getUnsafe();
+ return unsafe.getAddress(normalize(unsafe.getInt(object, 4L)) + 8L);
+ }
+
+ private static long normalize(int value) {
+ if (value >= 0) return value;
+ return (~0L >>> 32) & value;
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/OutOfMemoryException.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/OutOfMemoryException.java
new file mode 100644
index 0000000000..c770752626
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/OutOfMemoryException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.memory.util;
+
+import org.apache.hugegraph.HugeException;
+
+public class OutOfMemoryException extends HugeException {
+
+ public OutOfMemoryException(String message) {
+ super(message);
+ }
+
+ public OutOfMemoryException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public OutOfMemoryException(String message, Object... args) {
+ super(String.format(message, args));
+ }
+
+ public OutOfMemoryException(String message, Throwable cause, Object... args) {
+ super(String.format(message, args), cause);
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/RoundUtil.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/RoundUtil.java
new file mode 100644
index 0000000000..6f6b3aba82
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/RoundUtil.java
@@ -0,0 +1,50 @@
+/*
+ * 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.memory.util;
+
+public class RoundUtil {
+
+ private static final long MB = 1 << 20;
+ private static long ALIGNMENT = 8;
+
+ public static void setAlignment(long alignment) {
+ ALIGNMENT = alignment;
+ }
+
+ public static long sizeAlign(long size) {
+ long reminder = size % ALIGNMENT;
+ return reminder == 0 ? size : size + ALIGNMENT - reminder;
+ }
+
+ public static long roundDelta(long reservedSize, long delta) {
+ return quantizedSize(reservedSize + delta) - reservedSize;
+ }
+
+ private static long quantizedSize(long size) {
+ if (size < 16 * MB) {
+ return roundUp(size, MB);
+ }
+ if (size < 64 * MB) {
+ return roundUp(size, 4 * MB);
+ }
+ return roundUp(size, 8 * MB);
+ }
+
+ private static long roundUp(long size, long factor) {
+ return (size + factor - 1) / factor * factor;
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/SerializationRuntimeException.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/SerializationRuntimeException.java
new file mode 100644
index 0000000000..83ce7c043f
--- /dev/null
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/memory/util/SerializationRuntimeException.java
@@ -0,0 +1,25 @@
+/*
+ * 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.memory.util;
+
+public class SerializationRuntimeException extends RuntimeException {
+
+ public SerializationRuntimeException(Throwable e) {
+ super("Unexpected error occurs in serialization", e);
+ }
+}
diff --git a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/structure/HugeProperty.java b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/structure/HugeProperty.java
index cc96bd27b8..65472e148b 100644
--- a/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/structure/HugeProperty.java
+++ b/hugegraph-server/hugegraph-core/src/main/java/org/apache/hugegraph/structure/HugeProperty.java
@@ -31,7 +31,7 @@ public abstract class HugeProperty implements Property, GraphType {
protected final HugeElement owner;
protected final PropertyKey pkey;
- protected final V value;
+ protected V value;
public HugeProperty(HugeElement owner, PropertyKey pkey, V value) {
E.checkArgument(owner != null, "Property owner can't be null");
diff --git a/hugegraph-server/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties b/hugegraph-server/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties
index 52c81bef1d..b879d190a4 100644
--- a/hugegraph-server/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties
+++ b/hugegraph-server/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties
@@ -88,3 +88,9 @@ cassandra.password=
# WARNING: These raft configurations are deprecated, please use the latest version instead.
# raft.mode=false
+
+# memory management config
+#memory.mode=off-heap
+#memory.max_capacity=1073741824
+#memory.one_query_max_capacity=104857600
+#memory.alignment=8
diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryAllocateTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryAllocateTest.java
new file mode 100644
index 0000000000..1c8df5bba2
--- /dev/null
+++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryAllocateTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.core.memory;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.allocator.MemoryAllocator;
+import org.apache.hugegraph.memory.allocator.NettyMemoryAllocator;
+import org.apache.hugegraph.memory.allocator.OnHeapMemoryAllocator;
+import org.apache.hugegraph.util.Bytes;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.util.IllegalReferenceCountException;
+import io.netty.util.ReferenceCountUtil;
+
+public class MemoryAllocateTest {
+
+ private static MemoryAllocator nettyAllocator;
+ private static MemoryAllocator onHeapAllocator;
+ private static MemoryManager memoryManager;
+
+ @Before
+ public void setup() {
+ MemoryManager.getInstance().getCurrentOffHeapAllocatedMemoryInBytes().set(0);
+ MemoryManager.getInstance().getCurrentOnHeapAllocatedMemoryInBytes().set(0);
+ }
+
+ @BeforeClass
+ public static void beforeClass() {
+ nettyAllocator = new NettyMemoryAllocator(MemoryManager.getInstance());
+ onHeapAllocator = new OnHeapMemoryAllocator(MemoryManager.getInstance());
+ memoryManager = MemoryManager.getInstance();
+ }
+
+ @Test
+ public void testNettyAllocate() {
+ ByteBuf memoryBlock1 = (ByteBuf) nettyAllocator.tryToAllocate(Bytes.KB);
+ ByteBuf memoryBlock2 = (ByteBuf) nettyAllocator.tryToAllocate(Bytes.MB);
+ Assert.assertNotNull(memoryBlock1);
+ Assert.assertEquals(Bytes.KB, memoryBlock1.capacity());
+ Assert.assertNotNull(memoryBlock2);
+ Assert.assertEquals(Bytes.MB, memoryBlock2.capacity());
+ Assert.assertEquals(Bytes.KB + Bytes.MB,
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ ByteBuf memoryBlock3 = (ByteBuf) nettyAllocator.tryToAllocate(Bytes.GB);
+ Assert.assertNull(memoryBlock3);
+ memoryBlock3 = (ByteBuf) nettyAllocator.forceAllocate(Bytes.GB);
+ Assert.assertNotNull(memoryBlock3);
+ Assert.assertEquals(Bytes.GB, memoryBlock3.capacity());
+ }
+
+ @Test
+ public void testNettyDeallocate() {
+ ByteBuf buf = (ByteBuf) nettyAllocator.tryToAllocate(Bytes.KB);
+ Assert.assertNotNull(buf);
+ Assert.assertTrue(buf.isWritable());
+ Assert.assertEquals(buf.capacity(),
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ Assert.assertEquals(1, ReferenceCountUtil.refCnt(buf));
+ nettyAllocator.releaseMemoryBlock(buf);
+ Assert.assertThrows(IllegalReferenceCountException.class, buf::memoryAddress);
+ Assert.assertEquals(0, ReferenceCountUtil.refCnt(buf));
+ nettyAllocator.returnMemoryToManager(buf.capacity());
+ Assert.assertEquals(0,
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ }
+
+ @Test
+ public void testOnHeapAllocate() {
+ AtomicReference memoryBlock1 =
+ (AtomicReference) onHeapAllocator.tryToAllocate(Bytes.KB);
+ AtomicReference memoryBlock2 =
+ (AtomicReference) onHeapAllocator.tryToAllocate(Bytes.MB);
+ Assert.assertNotNull(memoryBlock1);
+ Assert.assertEquals(Bytes.KB, memoryBlock1.get().length);
+ Assert.assertNotNull(memoryBlock2);
+ Assert.assertEquals(Bytes.MB, memoryBlock2.get().length);
+ Assert.assertEquals(Bytes.KB + Bytes.MB,
+ memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().get());
+ AtomicReference memoryBlock3 =
+ (AtomicReference) onHeapAllocator.tryToAllocate(Bytes.GB);
+ Assert.assertNull(memoryBlock3);
+ memoryBlock3 = (AtomicReference) onHeapAllocator.forceAllocate(Bytes.GB);
+ Assert.assertNotNull(memoryBlock3);
+ Assert.assertEquals(Bytes.GB, memoryBlock3.get().length);
+ }
+
+ @Test
+ public void testOnHeapDeallocate() {
+ AtomicReference buf =
+ (AtomicReference) onHeapAllocator.tryToAllocate(Bytes.KB);
+ Assert.assertNotNull(buf);
+ Assert.assertEquals(buf.get().length,
+ memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().get());
+ onHeapAllocator.returnMemoryToManager(buf.get().length);
+ Assert.assertEquals(0,
+ memoryManager.getCurrentOnHeapAllocatedMemoryInBytes().get());
+ onHeapAllocator.releaseMemoryBlock(buf);
+ Assert.assertNull(buf.get());
+ }
+}
diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryConsumerTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryConsumerTest.java
new file mode 100644
index 0000000000..b3a83d0e50
--- /dev/null
+++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryConsumerTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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.core.memory;
+
+import java.nio.file.Paths;
+import java.util.UUID;
+
+import org.apache.commons.configuration2.PropertiesConfiguration;
+import org.apache.commons.configuration2.ex.ConfigurationException;
+import org.apache.commons.configuration2.io.FileHandler;
+import org.apache.hugegraph.HugeGraph;
+import org.apache.hugegraph.StandardHugeGraph;
+import org.apache.hugegraph.backend.cache.CachedBackendStore;
+import org.apache.hugegraph.backend.id.EdgeId;
+import org.apache.hugegraph.backend.id.Id;
+import org.apache.hugegraph.backend.id.IdGenerator;
+import org.apache.hugegraph.backend.query.Query;
+import org.apache.hugegraph.backend.serializer.BinaryBackendEntry;
+import org.apache.hugegraph.config.HugeConfig;
+import org.apache.hugegraph.dist.RegisterUtil;
+import org.apache.hugegraph.masterelection.GlobalMasterInfo;
+import org.apache.hugegraph.memory.consumer.OffHeapObject;
+import org.apache.hugegraph.memory.consumer.factory.IdFactory;
+import org.apache.hugegraph.memory.consumer.factory.PropertyFactory;
+import org.apache.hugegraph.memory.consumer.impl.id.StringIdOffHeap;
+import org.apache.hugegraph.schema.EdgeLabel;
+import org.apache.hugegraph.schema.PropertyKey;
+import org.apache.hugegraph.schema.SchemaManager;
+import org.apache.hugegraph.structure.HugeEdge;
+import org.apache.hugegraph.structure.HugeEdgeProperty;
+import org.apache.hugegraph.structure.HugeProperty;
+import org.apache.hugegraph.structure.HugeVertex;
+import org.apache.hugegraph.testutil.Assert;
+import org.apache.hugegraph.type.HugeType;
+import org.apache.hugegraph.type.define.Directions;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.util.IllegalReferenceCountException;
+
+public class MemoryConsumerTest extends MemoryManageTest {
+
+ private static HugeGraph graph;
+ private static boolean registered = false;
+
+ public static HugeGraph buildGraph() throws ConfigurationException {
+ String projectRoot = Paths.get("").toAbsolutePath().getParent().getParent().toString();
+ String CONF = projectRoot + "/hugegraph-commons/hugegraph-common/src/test/java/org/apache" +
+ "/hugegraph/unit/config/test.conf";
+ PropertiesConfiguration configuration = new PropertiesConfiguration();
+ FileHandler fileHandler = new FileHandler(configuration);
+ fileHandler.load(CONF);
+ HugeConfig config = new HugeConfig(configuration);
+ return new StandardHugeGraph(config);
+ }
+
+ @BeforeClass
+ public static void setup() throws ConfigurationException {
+ graph = buildGraph();
+ graph.clearBackend();
+ graph.initBackend();
+ graph.serverStarted(GlobalMasterInfo.master("server-test"));
+ if (registered) {
+ return;
+ }
+ RegisterUtil.registerBackends();
+ registered = true;
+
+ SchemaManager schema = graph.schema();
+ schema.propertyKey("name").asText().create();
+ schema.vertexLabel("book")
+ .properties("name")
+ .primaryKeys("name")
+ .enableLabelIndex(false)
+ .create();
+ }
+
+ @AfterClass
+ public static void clear() throws Exception {
+ if (graph == null) {
+ return;
+ }
+
+ try {
+ graph.clearBackend();
+ } finally {
+ graph.close();
+ graph = null;
+ }
+ }
+
+ @Test
+ public void testStringId() {
+ Id stringIdOffHeap = IdFactory.getInstance().newStringId("java");
+ Id stringId = IdGenerator.of("java");
+ Assert.assertNotNull(stringIdOffHeap);
+ Assert.assertEquals("java", stringIdOffHeap.asString());
+ Assert.assertEquals(stringId, ((OffHeapObject) stringIdOffHeap).zeroCopyReadFromByteBuf());
+ // Test release memoryBlock
+ memoryManager.getCorrespondingTaskMemoryPool(Thread.currentThread().getName())
+ .releaseSelf("test", false);
+ Assert.assertThrows(IllegalReferenceCountException.class,
+ ((StringIdOffHeap) stringIdOffHeap).getIdOffHeap()::memoryAddress);
+ }
+
+ @Test
+ public void testLongId() {
+ Id idOffHeap = IdFactory.getInstance().newLongId(1);
+ Id id = IdGenerator.of(1);
+ Assert.assertNotNull(idOffHeap);
+ Assert.assertEquals(1, idOffHeap.asLong());
+ Assert.assertEquals(id, ((OffHeapObject) idOffHeap).zeroCopyReadFromByteBuf());
+ }
+
+ @Test
+ public void testUuidId() {
+ UUID uuidEncoding = UUID.randomUUID();
+ Id idOffHeap = IdFactory.getInstance().newUuidId(uuidEncoding);
+ Id id = IdGenerator.of(uuidEncoding);
+ Assert.assertNotNull(idOffHeap);
+ Assert.assertArrayEquals(id.asBytes(), idOffHeap.asBytes());
+ Assert.assertEquals(id, ((OffHeapObject) idOffHeap).zeroCopyReadFromByteBuf());
+ }
+
+ @Test
+ public void testBinaryId() {
+ Id stringIdOffHeap = IdFactory.getInstance().newStringId("java");
+ Id idOffHeap =
+ IdFactory.getInstance().newBinaryId(stringIdOffHeap.asBytes(), stringIdOffHeap);
+ Id id = new BinaryBackendEntry.BinaryId(stringIdOffHeap.asBytes(), stringIdOffHeap);
+ Assert.assertNotNull(idOffHeap);
+ Assert.assertArrayEquals(stringIdOffHeap.asBytes(), idOffHeap.asBytes());
+ Assert.assertEquals(id, ((OffHeapObject) idOffHeap).zeroCopyReadFromByteBuf());
+ }
+
+ @Test
+ public void testObjectId() {
+ TestObject object = new TestObject();
+ object.x = 1;
+ object.y = "test";
+ Id idOffHeap =
+ IdFactory.getInstance().newObjectId(object);
+ Id id = new IdGenerator.ObjectId(object);
+ Assert.assertNotNull(idOffHeap);
+ Assert.assertEquals(id.asObject(), idOffHeap.asObject());
+ Assert.assertEquals(id, ((OffHeapObject) idOffHeap).zeroCopyReadFromByteBuf());
+ }
+
+ @Test
+ public void testQueryId() {
+ Query q = new Query(HugeType.VERTEX);
+ Id idOffHeap =
+ IdFactory.getInstance().newQueryId(q);
+ Id id = new CachedBackendStore.QueryId(q);
+ Assert.assertNotNull(idOffHeap);
+ Assert.assertEquals(id.toString(), idOffHeap.toString());
+ Assert.assertEquals(id, ((OffHeapObject) idOffHeap).zeroCopyReadFromByteBuf());
+ }
+
+ @Test
+ public void testEdgeId() {
+ Id stringIdOffHeap = IdFactory.getInstance().newStringId("java");
+ HugeVertex java = new HugeVertex(graph, stringIdOffHeap, graph.vertexLabel("book"));
+ Id edgeLabelIdOffHeap = IdFactory.getInstance().newLongId(1);
+ Id subLabelIdOffHeap = IdFactory.getInstance().newLongId(2);
+ Id edgeIdOffHeap =
+ IdFactory.getInstance().newEdgeId(java, Directions.OUT, edgeLabelIdOffHeap,
+ subLabelIdOffHeap,
+ "test", java);
+ Id edgeId = new EdgeId(java,
+ Directions.OUT,
+ (Id) ((OffHeapObject) edgeLabelIdOffHeap).zeroCopyReadFromByteBuf(),
+ (Id) ((OffHeapObject) subLabelIdOffHeap).zeroCopyReadFromByteBuf(),
+ "test",
+ java);
+ Assert.assertNotNull(edgeIdOffHeap);
+ // TODO: adopt equals method
+ Assert.assertEquals(edgeId.asString(), edgeIdOffHeap.asString());
+ }
+
+ @Test
+ public void testProperty() {
+ Id stringIdOffHeap = IdFactory.getInstance().newStringId("java");
+ HugeVertex java = new HugeVertex(graph, stringIdOffHeap, graph.vertexLabel("book"));
+ Id edgeLabelIdOffHeap = IdFactory.getInstance().newLongId(1);
+ Id subLabelIdOffHeap = IdFactory.getInstance().newLongId(2);
+
+ Id edgeId = new EdgeId(java,
+ Directions.OUT,
+ (Id) ((OffHeapObject) edgeLabelIdOffHeap).zeroCopyReadFromByteBuf(),
+ (Id) ((OffHeapObject) subLabelIdOffHeap).zeroCopyReadFromByteBuf(),
+ "test",
+ java);
+ HugeEdge testEdge = new HugeEdge(graph, edgeId, EdgeLabel.NONE);
+ PropertyKey propertyKey = new PropertyKey(null, IdFactory.getInstance().newLongId(3),
+ "fake");
+
+ String propertyValue = "test";
+ HugeProperty propertyOffHeap =
+ PropertyFactory.getInstance(String.class).newHugeEdgeProperty(testEdge,
+ propertyKey,
+ propertyValue);
+ HugeEdgeProperty property = new HugeEdgeProperty<>(testEdge,
+ propertyKey,
+ propertyValue);
+ Assert.assertNotNull(propertyOffHeap);
+ Assert.assertEquals(property.value(), propertyOffHeap.value());
+ Assert.assertEquals(property, propertyOffHeap);
+ }
+
+ static class TestObject {
+
+ long x;
+ String y;
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof TestObject)) {
+ return false;
+ }
+ TestObject other = (TestObject) obj;
+ return this.x == other.x && this.y.equals(other.y);
+ }
+ }
+}
diff --git a/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryManageTest.java b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryManageTest.java
new file mode 100644
index 0000000000..4855c09582
--- /dev/null
+++ b/hugegraph-server/hugegraph-test/src/main/java/org/apache/hugegraph/core/memory/MemoryManageTest.java
@@ -0,0 +1,282 @@
+/*
+ * 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.core.memory;
+
+import org.apache.hugegraph.memory.MemoryManager;
+import org.apache.hugegraph.memory.pool.MemoryPool;
+import org.apache.hugegraph.memory.pool.impl.TaskMemoryPool;
+import org.apache.hugegraph.memory.util.RoundUtil;
+import org.apache.hugegraph.testutil.Assert;
+import org.apache.hugegraph.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+
+/**
+ * This test will construct a query context with 2 query for testing memory management
+ *
+ * query1
+ * |
+ * --> task1
+ * |
+ * ----> operator1
+ * |
+ * ----> operator2
+ * |
+ * --> task2
+ * |
+ * ----> operator1
+ *
+ * query2
+ * |
+ * --> task1
+ * |
+ * ----> operator1
+ */
+public class MemoryManageTest {
+
+ protected static MemoryManager memoryManager;
+ protected static MemoryPool query1MemoryPool;
+ protected static MemoryPool query1Task1MemoryPool;
+ protected static String QUERY1_TASK1_THREAD_NAME = "main";
+ protected static MemoryPool query1Task2MemoryPool;
+ protected static String QUERY1_TASK2_THREAD_NAME = "QUERY1-THREAD-2";
+ protected static MemoryPool query1Task1Operator1MemoryPool;
+ protected static MemoryPool query1Task1Operator2MemoryPool;
+ protected static MemoryPool query1Task2Operator1MemoryPool;
+
+ protected static MemoryPool query2MemoryPool;
+ protected static MemoryPool query2Task1MemoryPool;
+ protected static String QUERY2_TASK1_THREAD_NAME = "QUERY2-THREAD-1";
+ protected static MemoryPool query2Task1Operator1MemoryPool;
+
+ @Before
+ public void setUp() {
+ memoryManager = MemoryManager.getInstance();
+ MemoryManager.setMemoryMode(MemoryManager.MemoryMode.ENABLE_OFF_HEAP_MANAGEMENT);
+ MemoryManager.setMaxMemoryCapacityInBytes(Bytes.GB);
+ MemoryManager.setMaxMemoryCapacityForOneQuery(Bytes.MB * 100);
+ RoundUtil.setAlignment(8);
+ query1MemoryPool = memoryManager.addQueryMemoryPool();
+ query2MemoryPool = memoryManager.addQueryMemoryPool();
+
+ query1Task1MemoryPool = query1MemoryPool.addChildPool("Task1");
+ memoryManager.bindCorrespondingTaskMemoryPool(QUERY1_TASK1_THREAD_NAME,
+ (TaskMemoryPool) query1Task1MemoryPool);
+ query1Task1Operator1MemoryPool = query1Task1MemoryPool.addChildPool("Operator1");
+ query1Task1Operator2MemoryPool = query1Task1MemoryPool.addChildPool("Operator2");
+
+ query1Task2MemoryPool = query1MemoryPool.addChildPool("Task2");
+ memoryManager.bindCorrespondingTaskMemoryPool(QUERY1_TASK2_THREAD_NAME,
+ (TaskMemoryPool) query1Task2MemoryPool);
+ query1Task2Operator1MemoryPool = query1Task2MemoryPool.addChildPool("Operator1");
+
+ query2Task1MemoryPool = query2MemoryPool.addChildPool("Task1");
+ memoryManager.bindCorrespondingTaskMemoryPool(QUERY2_TASK1_THREAD_NAME,
+ (TaskMemoryPool) query2Task1MemoryPool);
+ query2Task1Operator1MemoryPool = query2Task1MemoryPool.addChildPool("Operator1");
+ }
+
+ @After
+ public void after() {
+ memoryManager.gcQueryMemoryPool(query1MemoryPool);
+ memoryManager.gcQueryMemoryPool(query2MemoryPool);
+ memoryManager.getCurrentAvailableMemoryInBytes()
+ .set(MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES);
+ Assert.assertEquals(0, memoryManager.getCurrentQueryMemoryPools().size());
+ Assert.assertEquals(0, query1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query2MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query1Task1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query2Task1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query1Task1Operator2MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query1Task1Operator1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query2Task1Operator1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ }
+
+ @Test
+ public void testCurrentWorkingMemoryPool() {
+ TaskMemoryPool taskMemoryPool =
+ (TaskMemoryPool) memoryManager.getCorrespondingTaskMemoryPool(
+ Thread.currentThread()
+ .getName());
+ Assert.assertNotNull(taskMemoryPool);
+ Assert.assertNotNull(taskMemoryPool.getCurrentWorkingOperatorMemoryPool());
+ }
+
+ @Test
+ public void testMemoryPoolStructure() {
+ Assert.assertEquals(2, memoryManager.getCurrentQueryMemoryPools().size());
+ Assert.assertEquals(2, query1MemoryPool.getChildrenCount());
+ Assert.assertEquals(2, query1Task1MemoryPool.getChildrenCount());
+ Assert.assertEquals(1, query1Task2MemoryPool.getChildrenCount());
+ Assert.assertEquals(0, query1Task1Operator1MemoryPool.getChildrenCount());
+
+ Assert.assertEquals(1, query2MemoryPool.getChildrenCount());
+ Assert.assertEquals(1, query2Task1MemoryPool.getChildrenCount());
+ Assert.assertEquals(0, query2Task1Operator1MemoryPool.getChildrenCount());
+ }
+
+ @Test
+ public void testRequiringMemory() {
+ long requireBytes = Bytes.KB;
+ ByteBuf memoryBlock = (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes
+ , query1Task1Operator1MemoryPool);
+ Assert.assertNotNull(memoryBlock);
+ Assert.assertEquals(requireBytes, memoryBlock.capacity());
+ Assert.assertEquals(requireBytes,
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ Assert.assertEquals(
+ MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES - query1MemoryPool.getAllocatedBytes(),
+ memoryManager.getCurrentAvailableMemoryInBytes().get());
+ // will use reserved memory, not requiring memory through manager
+ ByteBuf memoryBlock2 =
+ (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes,
+ query1Task1Operator1MemoryPool);
+ Assert.assertNotNull(memoryBlock2);
+ Assert.assertEquals(requireBytes, memoryBlock2.capacity());
+ Assert.assertEquals(requireBytes * 2,
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ Assert.assertEquals(
+ MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES - query1MemoryPool.getAllocatedBytes(),
+ memoryManager.getCurrentAvailableMemoryInBytes().get());
+ }
+
+ @Test
+ public void testOOM() {
+ long requireBytes = Bytes.GB * 2;
+ ByteBuf memoryBlock = (ByteBuf) query2Task1Operator1MemoryPool.requireMemory(requireBytes
+ , query2Task1Operator1MemoryPool);
+ Assert.assertNull(memoryBlock);
+ }
+
+ @Test
+ public void testReleaseMemoryWithTask() {
+ long requireBytes = Bytes.KB;
+ ByteBuf memoryBlock = (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes
+ , query1Task1Operator1MemoryPool);
+ Assert.assertNotNull(memoryBlock);
+ Assert.assertEquals(requireBytes, memoryBlock.capacity());
+ memoryManager.getCorrespondingTaskMemoryPool(QUERY1_TASK1_THREAD_NAME)
+ .releaseSelf("Test release by hand", false);
+ Assert.assertNull(memoryManager.getCorrespondingTaskMemoryPool(QUERY1_TASK1_THREAD_NAME));
+ Assert.assertEquals(1, query1MemoryPool.getChildrenCount());
+ Assert.assertEquals(0, query1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query1MemoryPool.getUsedBytes());
+ Assert.assertEquals(0, query1Task1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query1Task1MemoryPool.getUsedBytes());
+ Assert.assertEquals(0, memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ Assert.assertEquals(MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES,
+ memoryManager.getCurrentAvailableMemoryInBytes().get());
+ }
+
+ @Test
+ public void testReleaseMemoryWithQuery() {
+ long requireBytes = Bytes.KB;
+ ByteBuf memoryBlock = (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes
+ , query1Task1Operator1MemoryPool);
+ Assert.assertNotNull(memoryBlock);
+ Assert.assertEquals(requireBytes, memoryBlock.capacity());
+ query1MemoryPool.releaseSelf("Test release by hand", true);
+ Assert.assertEquals(0, query1MemoryPool.getChildrenCount());
+ Assert.assertEquals(0, query1MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(0, query1MemoryPool.getUsedBytes());
+ Assert.assertEquals(0, memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ Assert.assertEquals(MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES,
+ memoryManager.getCurrentAvailableMemoryInBytes().get());
+ }
+
+ @Test
+ public void testExpandCapacity() {
+ long requireBytes = Bytes.KB;
+ ByteBuf memoryBlock = (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes
+ , query1Task1Operator1MemoryPool);
+ Assert.assertNotNull(memoryBlock);
+ Assert.assertEquals(requireBytes, memoryBlock.capacity());
+ long maxCapacity = Bytes.KB * 100;
+ query2MemoryPool.setMaxCapacityBytes(maxCapacity);
+ long requireBytes2 = maxCapacity * 2;
+ ByteBuf memoryBlock2 =
+ (ByteBuf) query2Task1Operator1MemoryPool.requireMemory(requireBytes2,
+ query2Task1Operator1MemoryPool);
+ Assert.assertNotNull(memoryBlock2);
+ Assert.assertEquals(requireBytes2, memoryBlock2.capacity());
+ Assert.assertEquals(requireBytes2 + requireBytes,
+ memoryManager.getCurrentOffHeapAllocatedMemoryInBytes().get());
+ Assert.assertEquals(
+ MemoryManager.MAX_MEMORY_CAPACITY_IN_BYTES - query1MemoryPool.getAllocatedBytes() -
+ query2MemoryPool.getAllocatedBytes(),
+ memoryManager.getCurrentAvailableMemoryInBytes().get());
+
+ }
+
+ @Test
+ public void testLocalArbitration() {
+ long totalMemory = 2 * Bytes.MB + Bytes.KB;
+ memoryManager.getCurrentAvailableMemoryInBytes().set(totalMemory);
+ long requireBytes = Bytes.KB;
+ // will allocate 2MB
+ ByteBuf memoryBlock = (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes
+ , query1Task1Operator1MemoryPool);
+ long requireBytes2 = Bytes.MB;
+ ByteBuf memoryBlock2 =
+ (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes2,
+ query1Task1Operator1MemoryPool);
+ Assert.assertEquals(Bytes.MB * 2, query1MemoryPool.getAllocatedBytes());
+ // query1 remaining 1023KB
+ Assert.assertEquals(Bytes.MB * 2 - requireBytes2 - requireBytes,
+ query1MemoryPool.getFreeBytes());
+ // totally remaining 1KB
+ Assert.assertEquals(totalMemory - 2 * Bytes.MB,
+ memoryManager.getCurrentAvailableMemoryInBytes().get());
+ // will try to allocate 1MB and trigger arbitration, which will fail and result in OOM
+ ByteBuf memoryBlock3 =
+ (ByteBuf) query2Task1Operator1MemoryPool.requireMemory(requireBytes,
+ query2Task1Operator1MemoryPool);
+ Assert.assertNull(memoryBlock3);
+ Assert.assertEquals(0, query2MemoryPool.getAllocatedBytes());
+ }
+
+ @Test
+ public void testGlobalArbitration() {
+ long totalMemory = 20 * Bytes.MB + Bytes.KB;
+ memoryManager.getCurrentAvailableMemoryInBytes().set(totalMemory);
+ long requireBytes = Bytes.MB * 17;
+ // will allocate 20MB
+ ByteBuf memoryBlock = (ByteBuf) query1Task1Operator1MemoryPool.requireMemory(requireBytes
+ , query1Task1Operator1MemoryPool);
+ Assert.assertEquals(Bytes.MB * 20, query1MemoryPool.getAllocatedBytes());
+ // query1 remaining 3MB
+ Assert.assertEquals(Bytes.MB * 3,
+ query1MemoryPool.getFreeBytes());
+ // totally remaining 1KB
+ Assert.assertEquals(Bytes.KB, memoryManager.getCurrentAvailableMemoryInBytes().get());
+ // will try to allocate 1MB and trigger arbitration, which will success
+ long requireBytes2 = Bytes.KB;
+ ByteBuf memoryBlock2 =
+ (ByteBuf) query2Task1Operator1MemoryPool.requireMemory(requireBytes2,
+ query2Task1Operator1MemoryPool);
+ Assert.assertNotNull(memoryBlock2);
+ Assert.assertEquals(Bytes.MB, query2MemoryPool.getAllocatedBytes());
+ Assert.assertEquals(Bytes.KB, memoryBlock2.capacity());
+ // totally still remain 1KB
+ Assert.assertEquals(Bytes.KB, memoryManager.getCurrentAvailableMemoryInBytes().get());
+ }
+}
diff --git a/install-dist/scripts/dependency/known-dependencies.txt b/install-dist/scripts/dependency/known-dependencies.txt
index 02b5dda11a..36255a96eb 100644
--- a/install-dist/scripts/dependency/known-dependencies.txt
+++ b/install-dist/scripts/dependency/known-dependencies.txt
@@ -1,3 +1,7 @@
+HdrHistogram-2.1.12.jar
+HdrHistogram-2.1.9.jar
+LatencyUtils-2.0.3.jar
+ST4-4.0.8.jar
accessors-smart-1.2.jar
airline-0.8.jar
android-json-0.0.20131108.vaadin1.jar
@@ -87,6 +91,7 @@ fastparse_2.12-2.0.4.jar
fastutil-8.5.9.jar
findbugs-annotations-1.3.9-1.jar
front-end-9.0-9.0.20190305.jar
+fury-core-0.9.0-SNAPSHOT.jar
gremlin-console-3.5.1.jar
gremlin-core-3.5.1.jar
gremlin-driver-3.5.1.jar
@@ -142,8 +147,6 @@ hamcrest-2.2.jar
hamcrest-core-1.3.jar
hanlp-portable-1.8.3.jar
hbase-shaded-endpoint-2.0.6.jar
-HdrHistogram-2.1.12.jar
-HdrHistogram-2.1.9.jar
hessian-3.3.6.jar
hessian-3.3.7.jar
hg-pd-client-1.5.0.jar
@@ -214,12 +217,12 @@ javassist-3.21.0-GA.jar
javassist-3.24.0-GA.jar
javassist-3.28.0-GA.jar
javatuples-1.2.jar
+javax-websocket-client-impl-9.4.46.v20220331.jar
+javax-websocket-server-impl-9.4.46.v20220331.jar
javax.activation-api-1.2.0.jar
javax.annotation-api-1.3.2.jar
javax.inject-1.jar
javax.json-1.0.jar
-javax-websocket-client-impl-9.4.46.v20220331.jar
-javax-websocket-server-impl-9.4.46.v20220331.jar
jaxb-api-2.3.1.jar
jaxb-core-3.0.2.jar
jaxb-impl-3.0.2.jar
@@ -262,8 +265,8 @@ jetty-util-9.4.46.v20220331.jar
jetty-util-ajax-9.4.46.v20220331.jar
jetty-webapp-9.4.46.v20220331.jar
jetty-xml-9.4.46.v20220331.jar
-jffi-1.2.16.jar
jffi-1.2.16-native.jar
+jffi-1.2.16.jar
jflex-1.8.2.jar
jieba-analysis-1.0.2.jar
jjwt-api-0.11.5.jar
@@ -280,10 +283,10 @@ jraft-core-1.3.11.jar
jraft-core-1.3.13.jar
jraft-core-1.3.9.jar
json-20210307.jar
-jsonassert-1.5.0.jar
json-path-2.5.0.jar
json-simple-1.1.jar
json-smart-2.3.jar
+jsonassert-1.5.0.jar
jsr305-3.0.1.jar
jsr305-3.0.2.jar
jul-to-slf4j-1.7.36.jar
@@ -314,7 +317,6 @@ 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
-LatencyUtils-2.0.3.jar
listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar
log4j-api-2.15.0.jar
log4j-api-2.17.0.jar
@@ -364,10 +366,10 @@ netty-buffer-4.1.52.Final.jar
netty-buffer-4.1.72.Final.jar
netty-codec-4.1.52.Final.jar
netty-codec-4.1.72.Final.jar
-netty-codec-http2-4.1.52.Final.jar
-netty-codec-http2-4.1.72.Final.jar
netty-codec-http-4.1.52.Final.jar
netty-codec-http-4.1.72.Final.jar
+netty-codec-http2-4.1.52.Final.jar
+netty-codec-http2-4.1.72.Final.jar
netty-codec-socks-4.1.52.Final.jar
netty-codec-socks-4.1.72.Final.jar
netty-common-4.1.52.Final.jar
@@ -415,20 +417,20 @@ powermock-module-junit4-2.0.0-RC.3.jar
powermock-module-junit4-common-2.0.0-RC.3.jar
powermock-module-junit4-rule-2.0.0-RC.3.jar
powermock-reflect-2.0.0-RC.3.jar
+proto-google-common-protos-1.17.0.jar
+proto-google-common-protos-2.0.1.jar
protobuf-java-3.11.0.jar
protobuf-java-3.17.2.jar
protobuf-java-3.21.7.jar
protobuf-java-3.5.1.jar
protobuf-java-util-3.17.2.jar
-proto-google-common-protos-1.17.0.jar
-proto-google-common-protos-2.0.1.jar
protostuff-api-1.6.0.jar
protostuff-collectionschema-1.6.0.jar
protostuff-core-1.6.0.jar
protostuff-runtime-1.6.0.jar
psjava-0.1.19.jar
-reporter-config3-3.0.3.jar
reporter-config-base-3.0.3.jar
+reporter-config3-3.0.3.jar
rewriting-9.0-9.0.20190305.jar
rocksdbjni-6.29.5.jar
rocksdbjni-7.2.2.jar
@@ -445,9 +447,9 @@ sjk-cli-0.22.jar
sjk-core-0.14.jar
sjk-core-0.22.jar
sjk-hflame-0.22.jar
+sjk-jfr-standalone-0.7.jar
sjk-jfr5-0.5.jar
sjk-jfr6-0.7.jar
-sjk-jfr-standalone-0.7.jar
sjk-json-0.14.jar
sjk-json-0.22.jar
sjk-nps-0.9.jar
@@ -491,7 +493,6 @@ spring-expression-5.3.20.jar
spring-jcl-5.3.20.jar
spring-web-5.3.20.jar
spring-webmvc-5.3.20.jar
-ST4-4.0.8.jar
stream-2.5.2.jar
swagger-annotations-1.5.18.jar
swagger-annotations-jakarta-2.2.18.jar