From 756fdf15a6402ab34ebb3a975196639b32d6729d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 6 Aug 2024 20:48:21 +0800 Subject: [PATCH 1/6] setup --- .../store/cli/StoreConsoleApplication.java | 6 +- .../store/cli/loader/HgThread2DB.java | 58 +++--- .../store/cli/scan/GrpcShardScanner.java | 2 +- .../org/apache/hugegraph/store/HgKvStore.java | 4 +- .../store/client/HgNodePartition.java | 4 +- .../client/HgStoreNodePartitionerImpl.java | 6 +- .../store/client/NodeTxExecutor.java | 4 +- .../store/client/NodeTxSessionProxy.java | 2 +- .../client/grpc/GrpcStoreStreamClient.java | 2 +- .../hugegraph/store/client/grpc/GrpcUtil.java | 4 +- .../store/client/grpc/KvBatchScanner.java | 49 ++--- .../client/grpc/KvBatchScannerMerger.java | 28 +-- .../org/apache/hugegraph/store/term/Bits.java | 4 +- .../hugegraph/store/HeartbeatService.java | 28 +-- .../apache/hugegraph/store/HgStoreEngine.java | 106 +++++------ .../hugegraph/store/PartitionEngine.java | 180 +++++++++--------- .../store/PartitionInstructionProcessor.java | 18 +- .../store/PartitionStateListener.java | 4 +- .../store/business/BusinessHandler.java | 10 +- .../store/business/BusinessHandlerImpl.java | 42 ++-- .../hugegraph/store/business/DataMover.java | 18 +- .../store/business/DefaultDataMover.java | 12 +- .../store/business/FilterIterator.java | 2 +- .../store/business/InnerKeyCreator.java | 2 +- .../store/business/InnerKeyFilter.java | 6 +- .../hugegraph/store/cmd/HgCmdClient.java | 10 +- .../hugegraph/store/cmd/HgCmdProcessor.java | 4 +- .../hugegraph/store/meta/GraphIdManager.java | 20 +- .../hugegraph/store/meta/GraphManager.java | 4 +- .../store/meta/MetadataKeyHelper.java | 2 +- .../hugegraph/store/meta/Partition.java | 2 +- .../store/meta/PartitionManager.java | 149 +++++++++------ .../hugegraph/store/meta/PartitionStats.java | 6 +- .../apache/hugegraph/store/meta/Shard.java | 2 +- .../hugegraph/store/meta/ShardGroup.java | 8 +- .../apache/hugegraph/store/meta/Store.java | 4 +- .../hugegraph/store/meta/StoreMetadata.java | 12 +- .../hugegraph/store/meta/TaskManager.java | 6 +- .../meta/asynctask/AbstractAsyncTask.java | 2 +- .../store/meta/asynctask/AsyncTask.java | 12 +- .../store/meta/asynctask/CleanTask.java | 2 +- .../store/meta/base/MetaStoreBase.java | 2 +- .../store/meta/base/PartitionMetaStore.java | 2 +- .../store/options/HgStoreEngineOptions.java | 28 +-- .../store/options/PartitionEngineOptions.java | 8 +- .../store/options/RaftRocksdbOptions.java | 2 +- .../hugegraph/store/pd/DefaultPdProvider.java | 12 +- .../store/pd/FakePdServiceProvider.java | 6 +- .../apache/hugegraph/store/pd/PdProvider.java | 2 +- .../store/raft/HgStoreStateMachine.java | 8 +- .../hugegraph/store/raft/RaftOperation.java | 10 +- .../hugegraph/store/raft/RaftTaskHandler.java | 2 +- .../store/snapshot/HgSnapshotHandler.java | 4 +- .../store/util/HgStoreException.java | 2 +- .../apache/hugegraph/store/util/IpUtil.java | 4 +- .../apache/hugegraph/store/util/Version.java | 4 +- 56 files changed, 487 insertions(+), 455 deletions(-) diff --git a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/StoreConsoleApplication.java b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/StoreConsoleApplication.java index 32949711f8..51e3c09b7e 100644 --- a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/StoreConsoleApplication.java +++ b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/StoreConsoleApplication.java @@ -52,7 +52,7 @@ public static void main(String[] args) { @Override public void run(String... args) throws IOException, InterruptedException, PDException { if (args.length <= 0) { - log.warn("参数类型 cmd[-load, -query, -scan]"); + log.warn("Parameter type cmd[-load, -query, -scan]"); } else { switch (args[0]) { case "-load": @@ -80,7 +80,7 @@ public void run(String... args) throws IOException, InterruptedException, PDExce break; case "-scan": if (args.length < 4) { - log.warn("参数类型 -scan pd graphName tableName"); + log.warn("Parameter type -scan pd graphName tableName"); } else { doScan(args[1], args[2], args[3]); } @@ -94,7 +94,7 @@ public void run(String... args) throws IOException, InterruptedException, PDExce scanner.getDataSingle(); break; default: - log.warn("参数类型错误,未执行任何程序"); + log.warn("Parameter type error, no program executed"); } } } diff --git a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java index 67a77d5831..1b9e0810ca 100644 --- a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java +++ b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java @@ -51,13 +51,13 @@ import lombok.extern.slf4j.Slf4j; /** - * 使用pd,支持raft - * 读取文件并多线程进行入库 + * Use pd, support raft + * Read files and perform multi-threaded storage processing. */ @Slf4j public class HgThread2DB { - /*正在进行和在排队的任务的总数*/ + /*Total number of tasks in progress and in queue*/ private static final AtomicInteger taskTotal = new AtomicInteger(0); private static final AtomicInteger queryTaskTotal = new AtomicInteger(0); private static final AtomicLong insertDataCount = new AtomicLong(); @@ -204,7 +204,7 @@ public boolean testOrder(String input) { } /** - * 多线程读取文件入库 + * Multithreaded file reading and storage into database * * @throws IOException * @throws InterruptedException @@ -216,14 +216,14 @@ public void startMultiprocessInsert(String filepath) throws IOException { MetricX metrics = null; long dataCount = 0; if (readfile.exists()) { - // 读取文件 + // Read file InputStreamReader isr = new InputStreamReader(new FileInputStream(readfile), StandardCharsets.UTF_8); BufferedReader reader = new BufferedReader(isr); String strLine = null; String tableName = HgCliUtil.TABLE_NAME; - // 积攒到多少个后执行线程入库,10万 + // Accumulate to how many threads before executing thread storage, 100,000 int maxlist = 100000; List keys = new ArrayList<>(maxlist); metrics = MetricX.ofStart(); @@ -232,7 +232,7 @@ public void startMultiprocessInsert(String filepath) throws IOException { keys.add(strLine); dataCount++; - // 读取文件中的10000条数据,启一个线程入库 + // Read 10000 pieces of data from the file, start a thread for data storage. if (dataCount % maxlist == 0) { List finalKeys = keys; Runnable task = () -> { @@ -266,7 +266,7 @@ public void startMultiprocessInsert(String filepath) throws IOException { isr.close(); reader.close(); - // 把剩余的入库 + // Move the remaining items into storage if (!keys.isEmpty()) { List finalKeys1 = keys; Runnable task = () -> { @@ -299,18 +299,18 @@ public void startMultiprocessInsert(String filepath) throws IOException { threadPool.shutdown(); } else { - System.out.println("样本文件不存在:" + filepath); + System.out.println("Sample file does not exist: " + filepath); } metrics.end(); log.info("*************************************************"); - log.info(" 主进程执行时间:" + metrics.past() / 1000 + "秒,一共执行:" + dataCount + "条"); + log.info(" Main process execution time: " + metrics.past() / 1000 + " seconds, total executed: " + dataCount + " items"); log.info("*************************************************"); - System.out.println(" 主进程执行时间 " + metrics.past() / 1000 + "秒"); - System.out.println("-----主进程执行结束---------"); + System.out.println(" Main process execution time " + metrics.past() / 1000 + " seconds"); + System.out.println("-----Main process execution ends---------"); } /** - * 多线程读取文件入库 + * Multithreaded file reading and storage into database * * @throws IOException * @throws InterruptedException @@ -324,7 +324,7 @@ public void autoMultiprocessInsert() throws IOException { String strLine = null; String tableName = HgCliUtil.TABLE_NAME; - // 积攒到多少个后执行线程入库,10万 + // Accumulate to how many to execute thread storage, 100,000 int maxlist = 100000; List keys = new ArrayList<>(maxlist); for (int x = 0; x < 10000000; x++) { @@ -371,20 +371,20 @@ public void autoMultiprocessInsert() throws IOException { metrics.end(); log.info("*************************************************"); - log.info(" 主进程执行时间:" + metrics.past() / 1000 + "秒,一共执行:" + dataCount + "条"); + log.info(" Main process execution time: " + metrics.past() / 1000 + " seconds, total executed: " + dataCount + " items"); log.info("*************************************************"); - System.out.println(" 主进程执行时间 " + metrics.past() / 1000 + "秒"); - System.out.println("-----主进程执行结束---------"); + System.out.println(" Main process execution time " + metrics.past() / 1000 + " seconds"); + System.out.println("-----Main process ends---------"); } public String getLong() { - //如果需要更长 或者更大冗余空间, 只需要 time * 10^n 即可 - //当前可保证1毫秒 生成 10000条不重复 + // If needed longer or more redundant space, just use time * 10^n + //Currently guaranteed to generate 10000 unique items in 1 millisecond. return String.format("%019x", longId.getAndIncrement()); } /** - * 执行查询,并将查询的结果做为下一次迭代的点放入队列 + * Execute the query, and put the results of the query into the queue as the point for the next iteration. */ private void queryAnd2Queue() { try { @@ -409,7 +409,7 @@ private void queryAnd2Queue() { HgKvEntry entry = iterator.next(); String newPoint = HgCliUtil.toStr(entry.value()); // log.info("query_key =" + newPoint); - // 统计查询次数 + // Statistical query times if (!newPoint.isEmpty() && hashSet.add(newPoint)) { queryCount.getAndIncrement(); totalQueryCount.getAndIncrement(); @@ -432,7 +432,7 @@ private void queryAnd2Queue() { } } } - // 达到1万个点后,去查询一次 + // After reaching 10,000 points, query once. if (newQueryList.size() > 10000 && listQueue.size() < 10000) { listQueue.put(newQueryList); insertQueueCount++; @@ -444,7 +444,7 @@ private void queryAnd2Queue() { } } } - // 一次查询如果不够1万,单独提交一次查询,确保所有的结果都能执行查询 + // If a query is less than 10,000, submit a separate query to ensure that all results can execute the query. if (!newQueryList.isEmpty() && listQueue.size() < 1000) { listQueue.put(newQueryList); } @@ -459,10 +459,10 @@ private void queryAnd2Queue() { } /** - * 多线程查询 + * Multithreaded query * - * @param point 起始查询点,后续根据这个点查询到的value做为下一次的查询条件进行迭代 - * @param scanCount 允许启动的线程数量 + * @param point Starting query point, subsequent queries will use the value obtained from this point as the next query condition for iteration. + * @param scanCount The number of threads allowed to start * @throws IOException * @throws InterruptedException */ @@ -559,10 +559,10 @@ public HgOwnerKey next() { metrics.end(); log.info("*************************************************"); - log.info(" 主进程执行时间:" + metrics.past() / 1000 + "秒; 查询:" + totalQueryCount.get() - + "次,qps:" + totalQueryCount.get() * 1000 / metrics.past()); + log.info(" Main process execution time: " + metrics.past() / 1000 + " seconds; Queries: " + totalQueryCount.get() + + "times, qps:" + totalQueryCount.get() * 1000 / metrics.past()); log.info("*************************************************"); - System.out.println("-----主进程执行结束---------"); + System.out.println("-----Main process ends---------"); } } diff --git a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/scan/GrpcShardScanner.java b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/scan/GrpcShardScanner.java index 29dbbc281c..e9e10829f0 100644 --- a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/scan/GrpcShardScanner.java +++ b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/scan/GrpcShardScanner.java @@ -130,7 +130,7 @@ public void onNext(ScanResponse value) { @Override public void onError(Throwable t) { - log.warn("调用grpc接口发生错误", t); + log.warn("Calling grpc interface encountered an error", t); latch.countDown(); } diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/HgKvStore.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/HgKvStore.java index db640592f3..f04e743f32 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/HgKvStore.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/HgKvStore.java @@ -43,8 +43,8 @@ public interface HgKvStore { boolean put(String table, HgOwnerKey ownerKey, byte[] value); /** - * 该版本被store内部使用。向分区写入数据, - * partitionId与key.keyCode必须与pd存储的分区信息保持一致。 + * This version is used internally by the store. Write data to the partition, + * partitionId and key.keyCode must be consistent with the partition information stored in pd. */ boolean directPut(String table, int partitionId, HgOwnerKey key, byte[] value); diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgNodePartition.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgNodePartition.java index 6fa354edec..57d2ebc16a 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgNodePartition.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgNodePartition.java @@ -27,10 +27,10 @@ public final class HgNodePartition { private final Long nodeId; - //当前key的hashcode + // Current key's hash code private final Integer keyCode; - //分区的开始结束范围 + // Partition start-end range private final Integer startKey; private final Integer endKey; private int hash = -1; diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgStoreNodePartitionerImpl.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgStoreNodePartitionerImpl.java index dba939ec86..606b279e8d 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgStoreNodePartitionerImpl.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/HgStoreNodePartitionerImpl.java @@ -48,7 +48,7 @@ public HgStoreNodePartitionerImpl(PDClient pdClient, HgStoreNodeManager nodeMana } /** - * 查询分区信息,结果通过HgNodePartitionerBuilder返回 + * Query partition information, the result is returned through HgNodePartitionerBuilder. */ @Override public int partition(HgNodePartitionerBuilder builder, String graphName, @@ -141,7 +141,7 @@ public int partition(HgNodePartitionerBuilder builder, String graphName, } /** - * 查询hgstore信息 + * Query hgstore information * * @return hgstore */ @@ -157,7 +157,7 @@ public HgStoreNode apply(String graphName, Long nodeId) { } /** - * 通知更新缓存 + * Notice to update cache */ @Override public int notice(String graphName, HgStoreNotice storeNotice) { diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxExecutor.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxExecutor.java index 01eea1af79..5dae57c4e0 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxExecutor.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxExecutor.java @@ -377,11 +377,11 @@ Optional retryingInvoke(Supplier supplier) { if (i + 1 <= NODE_MAX_RETRYING_TIMES) { try { int sleepTime; - // 前三次每隔一秒做一次尝试 + // The first three times try once every second if (i < 3) { sleepTime = 1; } else { - // 后面逐次递增 + // Subsequent incremental sleepTime = i - 1; } log.info("Waiting {} seconds " + diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java index 066f96893d..e6d5de8212 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java @@ -692,7 +692,7 @@ public boolean doAction(String table, HgOwnerKey startKey, Integer code, } private List toNodeTkvList(Builder scanReqBuilder) { - // TODO 使用builder获取owner + // TODO use builder to get owner String table = scanReqBuilder.getTable(); HgOwnerKey ownerKey = HgStoreClientConst.ALL_PARTITION_OWNER_KEY; byte[] allOwner = ownerKey.getOwner(); diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcStoreStreamClient.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcStoreStreamClient.java index 93cfe7acac..4e5735b32b 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcStoreStreamClient.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcStoreStreamClient.java @@ -199,7 +199,7 @@ KvCloseableIterator doBatchScan(HgStoreNodeSession nodeSession, HgScanQuery return KvBatchScanner5.scan(nodeSession, this.getStub(nodeSession), scanQuery); } - // 返回多个小的迭代器,允许上层并行处理 + // Return multiple small iterators, allowing upper-level parallel processing KvCloseableIterator> doBatchScan3(HgStoreNodeSession nodeSession, HgScanQuery scanQuery, KvCloseableIterator iterator) { diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java index 2191f7aee7..313629f1d0 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java @@ -63,7 +63,7 @@ static Key.Builder getOwnerKeyBuilder() { Key.Builder builder = keyBuilder.get(); if (builder == null) { builder = Key.newBuilder(); - // TODO 线程级变量,寻找删除时机 + // TODO Thread-level variables, find the timing for deletion keyBuilder.set(builder); } return builder; @@ -86,7 +86,7 @@ static Key toKey(HgOwnerKey ownerKey) { Key.Builder builder = keyBuilder.get(); if (builder == null) { builder = Key.newBuilder(); - // TODO 线程级变量,寻找删除时机 + // TODO Thread-level variables, find the timing for deletion keyBuilder.set(builder); } return builder diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java index dce65636be..9b4dff13e0 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java @@ -47,7 +47,7 @@ import lombok.extern.slf4j.Slf4j; /** - * 批量流式查询客户端实现类 + * Batch streaming query client implementation class *

* created on 2022/07/23 * @@ -59,9 +59,10 @@ public class KvBatchScanner implements Closeable { static final Supplier> NO_DATA = () -> null; static int maxTaskSizePerStore = PropertyUtil.getInt("net.kv.scanner.task.size", 8); - private final StreamObserver sender; // 命令发送器 - private final KvBatchScannerMerger notifier; // 数据通知 - private final String graphName; // 图名 + private final StreamObserver sender; // command sender + private final KvBatchScannerMerger notifier; // Data notification + 名 +private final String graphName; // graph name private final HgScanQuery scanQuery; private final ScanReceiptRequest.Builder responseBuilder = ScanReceiptRequest.newBuilder(); private final KvBatchReceiver receiver; @@ -82,12 +83,12 @@ public KvBatchScanner( receiver = new KvBatchReceiver(this, scanQuery.getOrderType() == ScanOrderType.ORDER_STRICT); sender = stub.scanBatch2(receiver); - sendQuery(this.scanQuery); // 发送查询请求 + sendQuery(this.scanQuery); // Send query request } /** - * 构建流式查询迭代器 - * scanQuery进行拆分,启动多个流式请求,提升store的并发性 + * Build streaming query iterators + * scanQuery is split to launch multiple streaming requests, enhancing the concurrency of the store. * * @param scanQuery scanQuery * @param handler task handler @@ -115,7 +116,7 @@ public static void scan( } /** - * 发送查询请求 + * Send query request * * @param query scan query */ @@ -133,7 +134,7 @@ public void sendQuery(HgScanQuery query) { } /** - * 发送应答 + * Send response */ public void sendResponse() { try { @@ -164,18 +165,18 @@ public void dataArrived(Supplier> supplier) throws } /** - * 数据接收结束 + * Data reception ended */ public void dataComplete() { close(); } - // 流被关闭 + // Flow is closed @Override public void close() { try { if (notifier.unregisterScanner(this) < 0) { - notifier.dataArrived(this, NO_DATA); // 任务结束,唤醒队列 + notifier.dataArrived(this, NO_DATA); // Task finished, wake up the queue } } catch (InterruptedException e) { log.error("exception ", e); @@ -192,7 +193,7 @@ public void close() { } /** - * 任务拆分器 + * Task Splitter */ static class TaskSplitter { @@ -200,9 +201,9 @@ static class TaskSplitter { final BiFunction taskHandler; private KvBatchScannerMerger notifier; private Iterator prefixItr; - private int maxTaskSize = 0; // 最大并行任务数 + private int maxTaskSize = 0; // maximum parallel task count private int maxBatchSize = PropertyUtil.getInt("net.kv.scanner.batch.size", 1000); - // 每批次最大点数量 + // Maximum points per batch private volatile boolean finished = false; private volatile boolean splitting = false; private volatile int nextKeySerialNo = 1; @@ -229,19 +230,19 @@ public boolean isFinished() { } /** - * 评估最大任务数 + * Evaluate maximum number of tasks */ private void evaluateMaxTaskSize() { - if (maxTaskSize == 0) { // 根据第一批次任务,得到store数量,然后计算最大任务数 + if (maxTaskSize == 0) { // According to the first batch of tasks, get the number of stores, and then calculate the maximum number of tasks if (scanQuery.getOrderType() == ScanOrderType.ORDER_STRICT) { - maxTaskSize = 1; // 点排序,每台机器一个流, 所有store流结束后才能启动其他流 + maxTaskSize = 1; // Point sorting, one stream per machine, all store streams must finish before starting other streams. } else { maxTaskSize = this.notifier.getScannerCount() * maxTaskSizePerStore; } - maxBatchSize = this.notifier.getScannerCount() * maxBatchSize; // 每台机器最多1000条 + maxBatchSize = this.notifier.getScannerCount() * maxBatchSize; // Each machine maximum 1000 items /* - * Limit少于10000时启动一个流,节省网络带宽 + * Limit fewer than 10000 to start a stream, save network bandwidth. */ if (scanQuery.getLimit() < maxBatchSize * 30L) { maxTaskSize = 1; @@ -250,7 +251,7 @@ private void evaluateMaxTaskSize() { } /** - * 拆分任务,任务拆分为多个grpc请求 + * Split task, task split into multiple gRPC requests */ public void splitTask() { if (this.finished || this.splitting) { @@ -271,10 +272,10 @@ public void splitTask() { taskHandler.apply( HgScanQuery.prefixOf(scanQuery.getTable(), keys, scanQuery.getOrderType()), this.notifier); - // 评估最大任务数 + // Evaluate maximum number of tasks evaluateMaxTaskSize(); if (this.notifier.getScannerCount() < this.maxTaskSize) { - splitTask(); // 未达到最大任务数,继续拆分 + splitTask(); // Not reached the maximum number of tasks, continue to split } } this.finished = !prefixItr.hasNext(); @@ -292,7 +293,7 @@ public synchronized void close() { } /** - * 查询结果接收器 + * Query Result Receiver */ static class KvBatchReceiver implements StreamObserver { diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScannerMerger.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScannerMerger.java index 4f666c973e..4f89a275c6 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScannerMerger.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScannerMerger.java @@ -36,8 +36,8 @@ import lombok.extern.slf4j.Slf4j; /** - * 批量查询结果归并,阻塞队列工作模式 - * 对请求任务的拆分,创建多个请求队列 + * Batch query result merging, blocking queue working mode + * Splitting of request tasks, creating multiple request queues */ @Slf4j public class KvBatchScannerMerger implements KvCloseableIterator>, @@ -69,14 +69,14 @@ public boolean hasNext() { int waitTime = 0; while (current == null) { try { - // 队列有数据,还有活跃的查询器,任务未分配完 + // Queue has data, and there are active queryers, tasks not yet allocated. if (queue.size() != 0 || scanners.size() > 0 || !taskSplitter.isFinished()) { - current = queue.poll(1, TimeUnit.SECONDS); //定期检查client是否被关闭了 + current = queue.poll(1, TimeUnit.SECONDS); // Regularly check if the client has been closed. } else { break; } if (current == null) { - // 超时重试 + // Timeout retry sendTimeout(); if (++waitTime > maxWaitCount) { log.error( @@ -125,7 +125,7 @@ public void registerScanner(KvBatchScanner closeable) { } /** - * 返回值<0表示任务结束 + * Return value < 0 indicates the task is finished. * * @param closeable * @return @@ -146,7 +146,7 @@ public int getScannerCount() { } /** - * 组装一个Scanner的多个有序迭代器为一个迭代器 + * Assemble multiple ordered iterators of a Scanner into one iterator */ static class ScannerDataQueue { @@ -170,7 +170,7 @@ public void add(Supplier> supplier) { } /** - * 迭代器是否有效,如果没有数据,等待数据到达 + * Whether the iterator is valid, if there is no data, wait for the data to arrive. * * @return */ @@ -179,7 +179,7 @@ public boolean hasNext() { try { int waitTime = 0; Supplier> current; - current = queue.poll(1, TimeUnit.SECONDS); //定期检查client是否被关闭了 + current = queue.poll(1, TimeUnit.SECONDS); // Regularly check if the client has been closed. if (current == null) { if (++waitTime > maxWaitCount) { break; @@ -221,18 +221,18 @@ private void moveNext() { } /** - * 对多个Scanner返回结果进行归并排序 + * Merge sort for multiple Scanner return results */ static class SortedScannerMerger extends KvBatchScannerMerger { - // 每一个流对应一个接收队列 + // Each stream corresponds to a receive queue private final Map scannerQueues = new ConcurrentHashMap<>(); public SortedScannerMerger(KvBatchScanner.TaskSplitter splitter) { super(splitter); queue.add(() -> { - // 对store返回结果进行归并排序 + // Perform merge sort on the store's return result return new HgKvIterator<>() { private ScannerDataQueue iterator; private int currentSN = 0; @@ -286,8 +286,8 @@ public HgKvEntry next() { } /** - * 从多个Scanner中挑选一个sn最小的迭代器 - * 如果Scanner没有数据,等待数据到达。 + * Pick an iterator with the smallest sn from multiple Scanners + * If Scanner has no data, wait for the data to arrive. * * @return */ diff --git a/hugegraph-store/hg-store-common/src/main/java/org/apache/hugegraph/store/term/Bits.java b/hugegraph-store/hg-store-common/src/main/java/org/apache/hugegraph/store/term/Bits.java index 2b78a22b8d..8433626b29 100644 --- a/hugegraph-store/hg-store-common/src/main/java/org/apache/hugegraph/store/term/Bits.java +++ b/hugegraph-store/hg-store-common/src/main/java/org/apache/hugegraph/store/term/Bits.java @@ -22,7 +22,7 @@ public class Bits { /** - * 大头字节序写入short + * Big-endian byte order write short */ public static void putShort(byte[] buf, int offSet, int x) { buf[offSet] = (byte) (x >> 8); @@ -37,7 +37,7 @@ public static void putInt(byte[] buf, int offSet, int x) { } /** - * 大头字节序读取short + * Big-endian byte order read short */ public static int getShort(byte[] buf, int offSet) { int x = buf[offSet] & 0xff; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java index 58a20cf468..6f2ca10985 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java @@ -47,8 +47,8 @@ @Slf4j public class HeartbeatService implements Lifecycle, PartitionStateListener { - private static final int MAX_HEARTBEAT_RETRY_COUNT = 5; // 心跳重试次数 - private static final int REGISTER_RETRY_INTERVAL = 1; //注册重试时间间隔,单位秒 + private static final int MAX_HEARTBEAT_RETRY_COUNT = 5; // Heartbeat retry count + private static final int REGISTER_RETRY_INTERVAL = 1; //Registration retry interval, in seconds private final HgStoreEngine storeEngine; private final List stateListeners; private final Object partitionThreadLock = new Object(); @@ -58,10 +58,10 @@ public class HeartbeatService implements Lifecycle, Partit private Store storeInfo; private Metapb.ClusterStats clusterStats; private StoreMetadata storeMetadata; - // 心跳失败次数 + // Heartbeat failure count private int heartbeatFailCount = 0; private int reportErrCount = 0; - // 线程休眠时间 + // Thread sleep time private volatile int timerNextDelay = 1000; private boolean terminated = false; @@ -116,14 +116,14 @@ public void setStoreMetadata(StoreMetadata storeMetadata) { this.storeMetadata = storeMetadata; } - // 集群是否准备就绪 + // Whether the cluster is ready public boolean isClusterReady() { return clusterStats.getState() == Metapb.ClusterState.Cluster_OK; } /** - * 服务状态有四种 - * 就绪,在线、离线、死亡(从集群排除) + * Service status has four types + * Ready, Online, Offline, Dead (excluded from the cluster) */ protected void doStoreHeartbeat() { while (!terminated) { @@ -170,7 +170,7 @@ protected void doPartitionHeartbeat() { protected void registerStore() { try { - // 注册 store,初次注册 PD 产生 id,自动给 storeinfo 赋值 + // Register store, initial registration of PD generates id, automatically assigns value to storeinfo this.storeInfo.setStoreAddress(IpUtil.getNearestAddress(options.getGrpcAddress())); this.storeInfo.setRaftAddress(IpUtil.getNearestAddress(options.getRaftAddress())); @@ -186,7 +186,7 @@ protected void registerStore() { } log.info("Register Store id= {} successfully. store = {}, clusterStats {}", storeInfo.getId(), storeInfo, this.clusterStats); - // 监听 partition 消息 + // Listen to partition messages pdProvider.startHeartbeatStream(error -> { onStateChanged(Metapb.StoreState.Offline); timerNextDelay = REGISTER_RETRY_INTERVAL * 1000; @@ -291,7 +291,7 @@ protected void partitionHeartbeat() { .getId()) .setRole(Metapb.ShardRole.Leader) .build(); - // 获取各个 shard 信息。 + // Get information for each shard. for (PartitionEngine partition : partitions) { Metapb.PartitionStats.Builder stats = Metapb.PartitionStats.newBuilder(); stats.setId(partition.getGroupId()); @@ -302,10 +302,10 @@ protected void partitionHeartbeat() { stats.addAllShard(partition.getShardGroup().getMetaPbShard()); - // shard 状态 + // shard status List shardStats = new ArrayList<>(); Map aliveShards = partition.getAlivePeers(); - // 统计 shard 状态 + // Statistics shard status partition.getShardGroup().getShards().forEach(shard -> { Metapb.ShardState state = Metapb.ShardState.SState_Normal; if (!aliveShards.containsKey(shard.getStoreId())) { @@ -322,7 +322,7 @@ protected void partitionHeartbeat() { statsList.add(stats.build()); } - // 发送心跳 + // Send heartbeat if (statsList.size() > 0) { pdProvider.partitionHeartbeat(statsList); } @@ -359,7 +359,7 @@ public void shutdown() { @Override public void partitionRoleChanged(Partition partition, PartitionRole newRole) { if (newRole == PartitionRole.LEADER) { - // leader 发生改变,激活心跳 + // leader changed, activate heartbeat synchronized (partitionThreadLock) { partitionThreadLock.notifyAll(); } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java index 2bb1f4842c..72a8becc7f 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java @@ -70,7 +70,7 @@ public class HgStoreEngine implements Lifecycle, HgStoreSt private final static HgStoreEngine instance = new HgStoreEngine(); private static ConcurrentHashMap engineLocks = new ConcurrentHashMap<>(); - // 分区raft引擎,key为GraphName_PartitionID + // Partition raft engine, key is GraphName_PartitionID private final Map partitionEngines = new ConcurrentHashMap<>(); private RpcServer rpcServer; private HgStoreEngineOptions options; @@ -87,9 +87,9 @@ public static HgStoreEngine getInstance() { } /** - * 1、读取StoreId,向pd注册,初次注册由PD生成StoreId,存储到本地 - * 2、注册成功,启动raft服务 - * 3、定时发送Store心跳和Partition心跳,与PD保持联系 + * 1. Read StoreId, register with PD, the StoreId is generated by PD for the first registration and stored locally. + * 2. Registration successful, start the raft service. + * 3. Timely send Store heartbeats and Partition heartbeats to maintain contact with PD. * * @param opts * @return @@ -118,19 +118,19 @@ public synchronized boolean init(final HgStoreEngineOptions opts) { partitionManager.addPartitionChangedListener(new PartitionChangedListener()); businessHandler = new BusinessHandlerImpl(partitionManager); - // 需要businessHandler 初始化后 + // Need businessHandler initialization afterwards partitionManager.load(); rpcServer = createRaftRpcServer(opts.getRaftAddress()); hgCmdClient = new HgCmdClient(); hgCmdClient.init(new RpcOptions(), (graphName, ptId) -> { - // 分裂的时候,还未及时的上报pd + // When splitting, it has not been reported to pd in time. if (getPartitionEngine(ptId) != null) { return getPartitionEngine(ptId).waitForLeader( options.getWaitLeaderTimeout() * 1000); } else { - // 可能出现跨分区的迁移 + // May occur cross-partition migration Metapb.Shard shard = pdProvider.getPartitionLeader(graphName, ptId); return JRaftUtils.getEndPoint( pdProvider.getStoreByID(shard.getStoreId()).getRaftAddress()); @@ -153,7 +153,7 @@ public synchronized boolean init(final HgStoreEngineOptions opts) { } /** - * 创建raft rpc server,用于store之间通讯 + * Create raft rpc server, for communication between stores */ private RpcServer createRaftRpcServer(String raftAddr) { Endpoint endpoint = JRaftUtils.getEndPoint(raftAddr); @@ -193,13 +193,13 @@ public void snapshotForTest() { } /** - * Store注册状态发生改变 + * Store registration status has changed */ @Override public void stateChanged(Store store, Metapb.StoreState oldState, Metapb.StoreState newState) { log.info("stateChanged, oldState {}, newState {}", oldState, newState); if (newState == Metapb.StoreState.Up) { - // 状态变为上线,记录store信息 + // Status changes to online, record store information partitionManager.setStore(store); partitionManager.loadPartition(); restoreLocalPartitionEngine(); @@ -207,12 +207,12 @@ public void stateChanged(Store store, Metapb.StoreState oldState, Metapb.StoreSt } /** - * 恢复本地的PartitionEngine,恢复PD返回的分区信息 - * 1、需要检查本次保存的分区,删除作废的分区 + * Restore the local PartitionEngine, restore the partition information returned by PD + * 1. Need to check the partition saved this time, delete the invalid partitions. */ public void restoreLocalPartitionEngine() { try { - if (!options.isFakePD()) { // FakePD模式不需要同步 + if (!options.isFakePD()) { // FakePD mode does not require synchronization partitionManager.syncPartitionsFromPD(partition -> { log.warn( "The local partition information is inconsistent with the PD server. " + @@ -235,9 +235,9 @@ public void restoreLocalPartitionEngine() { } /** - * 收到store raft addr 变更,需要重新创建raft group + * Received store raft addr change, need to recreate raft group * - * @param storeId 变更的store id + * @param storeId Changed store id */ public void rebuildRaftGroup(long storeId) { partitionEngines.forEach((partId, engine) -> { @@ -266,7 +266,7 @@ public void rebuildRaftGroup(long storeId) { } /** - * 创建 raft Node + * Create raft Node * * @param partition * @return @@ -288,7 +288,7 @@ private PartitionEngine createPartitionEngine(int groupId, ShardGroup shardGroup if ((engine = partitionEngines.get(groupId)) == null) { engineLocks.computeIfAbsent(groupId, k -> new Object()); synchronized (engineLocks.get(groupId)) { - // 分区分裂时特殊情况(集群中图分区数量不一样),会导致分裂的分区,可能不在本机器上. + // Special cases during partition splitting (different number of graph partitions in the cluster) can cause the splitting partition not to be on this machine. if (conf != null) { var list = conf.listPeers(); list.addAll(conf.listLearners()); @@ -325,26 +325,26 @@ private PartitionEngine createPartitionEngine(int groupId, ShardGroup shardGroup ptOpts.setRaftDataPath(partitionManager.getRaftDataPath(groupId)); ptOpts.setRaftSnapShotPath(partitionManager.getRaftSnapShotPath(groupId)); ptOpts.setRaftOptions(options.getRaftOptions()); - // raft任务处理器 + // raft task processor ptOpts.setTaskHandler(options.getTaskHandler()); - // 分区状态监听 + // Partition status listening engine.addStateListener(this.heartbeatService); engine.init(ptOpts); partitionEngines.put(ptOpts.getGroupId(), engine); } } } - // 检查是否活跃,如果不活跃,则重新创建 + // Check if active, if not active, recreate engine.checkActivity(); return engine; } /** - * 创建 raft分组,除了创建本地raft node,还要通知其他peer创建raft node - * 1、遍历partition.shards - * 2、根据storeId获取Store信息 - * 3、建立向其他store的raft rpc,发送StartRaft消息 + * Create raft group, in addition to creating the local raft node, also need to notify other peers to create raft nodes. + * 1. Traverse partition.shards + Information by storeId + * 3. Establish Raft RPC to other stores, send StartRaft messages. * * @param partition * @return @@ -359,13 +359,13 @@ public PartitionEngine createPartitionGroups(Partition partition) { var shardGroup = partitionManager.getShardGroup(partition.getId()); if (shardGroup != null) { - // raft不存在,通知follower创建raft + // raft not exist, notify follower to create raft shardGroup.getShards().forEach((shard) -> { Store store = partitionManager.getStore(shard.getStoreId()); if (store == null || partitionManager.isLocalStore(store)) { return; } - // 向其他peer发消息,创建raft 分组。此处是异步发送 + // Send messages to other peers, create raft groups. This is an asynchronous send. hgCmdClient.createRaftNode(store.getRaftAddress(), List.of(partition), status -> { log.info( @@ -377,7 +377,7 @@ public PartitionEngine createPartitionGroups(Partition partition) { }); } } else { - // raft存在,修改分区列表,通过raft同步给follower + // raft exists, modify partition list, synchronize to follower through raft engine = createPartitionEngine(partition); } return engine; @@ -391,7 +391,7 @@ public void destroyPartitionGroups(Partition partition) { if (store == null) { return; } - // 向其他peer发消息,创建raft 分组。此处是异步发送 + // Send messages to other peers, create raft groups. This is an asynchronous send. hgCmdClient.destroyRaftNode(store.getRaftAddress(), Arrays.asList(new Partition[]{partition}), status -> { @@ -406,7 +406,7 @@ public void destroyPartitionGroups(Partition partition) { } /** - * 停止分区,并销毁数据 + * Stop partitioning, and destroy data */ public synchronized void destroyPartitionEngine(Integer groupId, List graphNames) { log.info("Partition {} start to be destroyed", groupId); @@ -416,14 +416,14 @@ public synchronized void destroyPartitionEngine(Integer groupId, List gr PartitionEngine ptEngine = partitionEngines.get(groupId); graphNames.forEach(graphName -> { ptEngine.removePartition(graphName); - // 删除数据 + // Delete data businessHandler.deletePartition(graphName, groupId); }); if (ptEngine.getPartitions().size() == 0) { ptEngine.destroy(); partitionEngines.remove(groupId); - // 删除对应的db文件夹 + // Delete the corresponding db folder businessHandler.destroyGraphDB(graphNames.get(0), groupId); } else { graphNames.forEach(graphName -> { @@ -434,7 +434,7 @@ public synchronized void destroyPartitionEngine(Integer groupId, List gr } /** - * 删除图数据,删除本地数据,并删除PD上的分区信息 + * Delete graph data, delete local data, and delete partition information on PD. */ public void deletePartition(Integer groupId, String graphName) { log.info("Partition {}-{} deletePartition", graphName, groupId); @@ -443,9 +443,9 @@ public void deletePartition(Integer groupId, String graphName) { } PartitionEngine ptEngine = partitionEngines.get(groupId); ptEngine.removePartition(graphName); - // 删除数据 + // Delete data businessHandler.deletePartition(graphName, groupId); - //通知PD删除分区数据 + //Notify PD to delete partition data if (ptEngine.isLeader()) { synchronized (this) { partitionManager.deletePartition(graphName, groupId); @@ -454,7 +454,7 @@ public void deletePartition(Integer groupId, String graphName) { } /** - * 获取所有的leader分区 + * Get all leader partitions * * @return */ @@ -469,7 +469,7 @@ public List getLeaderPartition() { } /** - * 获取分区所有活跃的peer + * Get all active peers in the partition * * @return */ @@ -486,7 +486,7 @@ public Map getAlivePeers(int groupId) { } /** - * 获取分区的最后提交的日志id + * Get the last committed log id of the partition * * @param groupId * @return @@ -546,14 +546,14 @@ public List getDataLocations() { } /** - * 添加raft任务 - * 1、检查partition是否存在 - * 1.1、如果不存在,则向PD查询分区是否属于本地 - * 1.1.1 如果分区属于本地,则创建raft分组,并通知其他Store - * 1.1.2 如果分区不属于本地,则抛出异常 - * 1.2 检查Partition是否是leader - * 1.2.1 如果是leader,则提交任务 - * 1.2.2 否则,返回错误 + * Add raft task + * 1. Check if the partition exists + * 1.1 If not exist, query PD to determine if the partition is local. + * 1.1.1 If the partition is local, create a raft group and notify other Stores. + * 1.1.2 If the partition is not local, throw an exception. + * 1.2 Check if Partition is the leader + * 1.2.1 If it is the leader, then submit the task. + * 1.2.2 Otherwise, return an error. * * @param partId * @param operation @@ -569,7 +569,7 @@ public void addRaftTask(String graphName, Integer partId, RaftOperation operatio Partition partition = partitionManager.findPartition(graphName, partId); if (partition != null) { engine = this.createPartitionGroups(partition); - // 可能迁移,不应该创建, 放到 synchronize体中,避免后面的 + // May migrate, should not create, put in synchronize block, avoid subsequent ones. if (engine != null) { engine.waitForLeader(options.getWaitLeaderTimeout() * 1000); } @@ -579,22 +579,22 @@ public void addRaftTask(String graphName, Integer partId, RaftOperation operatio } if (engine != null) { - // 等待Leader + // Waiting for Leader Endpoint leader = engine.waitForLeader(options.getWaitLeaderTimeout() * 1000); if (engine.isLeader()) { engine.addRaftTask(operation, closure); } else if (leader != null) { - // 当前不是leader,返回leader所在的storeId + // Currently not the leader, return the storeId where the leader is located. Store store = partitionManager.getStoreByRaftEndpoint(engine.getShardGroup(), leader.toString()); if (store.getId() == 0) { - // 本地未找到Leader的Store信息,可能Partition还未同步过来,重新向Leader获取。 + // Local store information for the Leader was not found, possibly the Partition has not been synchronized yet, reacquire from the Leader. Store leaderStore = hgCmdClient.getStoreInfo(leader.toString()); store = leaderStore != null ? leaderStore : store; log.error("getStoreByRaftEndpoint error store:{}, shard: {}, leader is {}", store, engine.getShardGroup().toString(), leader); } - // Leader 不是本机,通知客户端 + // Leader is not local, notify client closure.onLeaderChanged(partId, store.getId()); closure.run(new Status(HgRaftError.NOT_LEADER.getNumber(), String.format("Partition %s-%d leader changed to %x", @@ -648,7 +648,7 @@ public int getRaftGroupCount() { } /** - * 监听rocksdb事件 + * Listen to rocksdb events * * @return */ @@ -673,7 +673,7 @@ public void onCompacted(String dbName) { class PartitionChangedListener implements PartitionManager.PartitionChangedListener { /** - * Partition对象发生改变,leader通知到其他的follower + * Partition object changes, leader notifies other followers. */ @Override public void onChanged(Partition partition) { @@ -702,7 +702,7 @@ public void run(Status status) { } /** - * Partition对象key范围、状态发生改变,通过主动寻找leader再通知到其他的follower + * Partition object key range, status changes, notify other followers by actively finding the leader. */ @Override public UpdatePartitionResponse rangeOrStateChanged(UpdatePartitionRequest request) { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java index f82fd77d7f..dd693a9224 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java @@ -144,7 +144,7 @@ public PartitionEngine(HgStoreEngine storeEngine, ShardGroup shardGroup) { // } /** - * 记录使用本raft的分区信息 + * Record the partition information using this raft. */ public synchronized void loadPartitionFromSnapshot(Partition partition) { @@ -190,7 +190,7 @@ public synchronized boolean init(PartitionEngineOptions opts) { } this.stateMachine.addTaskHandler(new TaskHandler()); - // 监听分组leader是否发生改变 + // Listen for changes in the group leader this.stateMachine.addStateListener(this); new File(options.getRaftDataPath()).mkdirs(); @@ -202,15 +202,15 @@ public synchronized boolean init(PartitionEngineOptions opts) { initConf.parse(peersList); } - // 设置Node参数,包括日志存储路径和状态机实例 + // Set Node parameters, including log storage path and state machine instance NodeOptions nodeOptions = new NodeOptions(); nodeOptions.setFsm(stateMachine); nodeOptions.setEnableMetrics(true); - // 日志路径 + // Log path nodeOptions.setLogUri(options.getRaftDataPath() + "/log"); - // raft元数据路径 + // raft metadata path nodeOptions.setRaftMetaUri(options.getRaftDataPath() + "/meta"); - // 快照路径 + // Snapshot path nodeOptions.setSnapshotUri(options.getRaftSnapShotPath() + "/snapshot"); nodeOptions.setSharedTimerPool(true); nodeOptions.setSharedElectionTimer(true); @@ -229,9 +229,9 @@ public LogStorage createLogStorage(final String uri, final RaftOptions raftOptio } } }); - // 初始集群 + // Initial cluster nodeOptions.setInitialConf(initConf); - // 快照时间间隔 + // Snapshot interval nodeOptions.setSnapshotIntervalSecs(options.getRaftOptions().getSnapshotIntervalSecs()); //nodeOptions.setSnapshotLogIndexMargin(options.getRaftOptions() @@ -242,7 +242,7 @@ public LogStorage createLogStorage(final String uri, final RaftOptions raftOptio nodeOptions.setRpcInstallSnapshotTimeout( options.getRaftOptions().getRpcInstallSnapshotTimeout()); nodeOptions.setElectionTimeoutMs(options.getRaftOptions().getElectionTimeoutMs()); - // 设置raft配置 + // Set raft configuration RaftOptions raftOptions = nodeOptions.getRaftOptions(); raftOptions.setDisruptorBufferSize(options.getRaftOptions().getDisruptorBufferSize()); raftOptions.setMaxEntriesSize(options.getRaftOptions().getMaxEntriesSize()); @@ -253,16 +253,16 @@ public LogStorage createLogStorage(final String uri, final RaftOptions raftOptio final PeerId serverId = JRaftUtils.getPeerId(options.getRaftAddress()); - // 构建raft组并启动raft + // Build raft group and start raft this.raftGroupService = new RaftGroupService(raftPrefix + options.getGroupId(), serverId, nodeOptions, storeEngine.getRaftRpcServer(), true); this.raftNode = raftGroupService.start(false); this.raftNode.addReplicatorStateListener(new ReplicatorStateListener()); - // 检查pd返回的peers是否和本地一致,如果不一致,重置peerlist + // Check if the peers returned by pd are consistent with the local ones, if not, reset the peerlist if (this.raftNode != null) { - //TODO 检查peer列表,如果peer发生改变,进行重置 + //TODO Check peer list, if peer changes, perform reset started = true; } @@ -280,22 +280,22 @@ public ShardGroup getShardGroup() { } /** - * 1、收到PD发送的分区迁移指令,向状态机添加迁移任务,状态为新建 - * 2、执行状态机消息,添加到任务队列,并执行任务 - * 3、比较新旧peer,查找出新增和删除的peer - * 4、如果存在新增的peer - * 4.1、对于新增的peer,通知peer创建raft状态机 - * 4.2、以learner方式加入raft group - * 4.3、监听snapshot同步事件,重复执行步骤3 - * 5、不存在新增的peer - * 5.1、移除learner,并等待返回 - * 5.2、修改learner为peer,加入raft group - * 6、存在被删除的peer - * 6.1、通知peer,删除状态机并删除数据 + * 1. Receive the partition migration command sent by PD, add the migration task to the state machine, the state is new. + * 2, execute state machine messages, add to the task queue, and execute tasks. + * 3. Compare old and new peers to identify added and removed peers. + * 4. If there is a new peer added + * 4.1, For newly added peers, notify the peer to create the raft state machine. + * 4.2, Join the raft group in learner mode. + * 4.3, Listen for snapshot synchronization events, and repeat step 3. + * 5. No new peers exist. + * 5.1, Remove learner and wait for return + * 5.2, Modify learner to peer, join raft group + * 6. Existence of deleted peer + * 6.1, Notify peer, delete state machine and delete data * * @param peers * @param done - * @return true表示完成,false表示未完成 + * @return true means completed, false means not completed */ public Status changePeers(List peers, final Closure done) { if (ListUtils.isEqualList(peers, RaftUtils.getPeerEndpoints(raftNode))) { @@ -306,14 +306,14 @@ public Status changePeers(List peers, final Closure done) { List oldPeers = RaftUtils.getAllEndpoints(raftNode); log.info("Raft {} changePeers start, old peer is {}, new peer is {}", getGroupId(), oldPeers, peers); - // 检查需要新增的peer。 + // Check the peer that needs to be added. List addPeers = ListUtils.removeAll(peers, oldPeers); - // 需要删除的learner。可能peer发生改变 + // learner to be deleted. Possible peer change. List removedPeers = ListUtils.removeAll(RaftUtils.getLearnerEndpoints(raftNode), peers); HgCmdClient rpcClient = storeEngine.getHgCmdClient(); - // 生成新的Configuration对象 + // Generate a new Configuration object Configuration oldConf = getCurrentConf(); Configuration conf = oldConf.copy(); if (!addPeers.isEmpty()) { @@ -327,7 +327,7 @@ public Status changePeers(List peers, final Closure done) { FutureClosure closure = new FutureClosure(addPeers.size()); addPeers.forEach(peer -> Utils.runInThread(() -> { - // 1. 创建新peer的raft对象 + // 1. Create a new peer's raft object rpcClient.createRaftNode(peer, partitionManager.getPartitionList(getGroupId()), conf, status -> { closure.run(status); @@ -339,7 +339,7 @@ public Status changePeers(List peers, final Closure done) { })); closure.get(); } else { - // 3. 检查learner是否完成快照同步 + // 3. Check if learner has completed snapshot synchronization boolean snapshotOk = true; for (PeerId peerId : raftNode.listLearners()) { Replicator.State state = getReplicatorState(peerId); @@ -350,7 +350,7 @@ public Status changePeers(List peers, final Closure done) { log.info("Raft {} {} getReplicatorState {}", getGroupId(), peerId, state); } if (snapshotOk && !conf.listLearners().isEmpty()) { - // 4. 删除learner,以peer重新加入 + // 4. Delete learner, rejoin as peer FutureClosure closure = new FutureClosure(); raftNode.removeLearners(conf.listLearners(), closure); if (closure.get().isOk()) { @@ -360,17 +360,17 @@ public Status changePeers(List peers, final Closure done) { }); result = Status.OK(); } else { - // 失败了重试 + // Failed, retrying result = HgRaftError.TASK_ERROR.toStatus(); } } else if (snapshotOk) { - result = Status.OK(); // 没有learner,说明只做删除操作 + result = Status.OK(); // No learner, indicating only delete operations are performed. } } if (result.isOk()) { - // 同步完成,删除旧peer + // Sync completed, delete old peer removedPeers.addAll(ListUtils.removeAll(oldPeers, peers)); - // 检查leader是否被删除,如果是,先进行leader迁移 + // Check if leader is deleted, if so, perform leader migration first. if (removedPeers.contains( this.getRaftNode().getNodeId().getPeerId().getEndpoint().toString())) { @@ -385,7 +385,7 @@ public Status changePeers(List peers, final Closure done) { var status = this.raftNode.transferLeadershipTo(PeerId.ANY_PEER); log.info("Raft {} transfer leader status : {}", getGroupId(), status); - // 需要重新发送指令给新leader + // Need to resend the command to the new leader return HgRaftError.TASK_ERROR.toStatus(); } } @@ -398,19 +398,19 @@ public Status changePeers(List peers, final Closure done) { } if (!RaftUtils.configurationEquals(oldConf, conf)) { - // 2. 新的peer以learner身份加入 - // 5. peer切换,增加新的peer,删除旧的peer + // 2. The new peer joins as a learner. + // 5. peer switching, add new peer, delete old peer FutureClosure closure = new FutureClosure(); raftNode.changePeers(conf, closure); if (closure.get().isOk()) { if (!removedPeers.isEmpty()) { removedPeers.forEach(peer -> Utils.runInThread(() -> { - // 6. 停止已被删除的peer + // 6. Stop the deleted peer rpcClient.destroyRaftNode(peer, partitionManager.getPartitionList(getGroupId()), status -> { if (!status.isOk()) { - // TODO 失败了怎么办? + // TODO What if it fails? log.error("Raft {} destroy node {}" + " error {}", options.getGroupId(), peer, @@ -420,7 +420,7 @@ public Status changePeers(List peers, final Closure done) { })); } } else { - // 失败了重试 + // Failed, retrying result = HgRaftError.TASK_ERROR.toStatus(); } log.info("Raft {} changePeers result {}, conf is {}", @@ -462,7 +462,7 @@ public void shutdown() { } /** - * 重启raft引擎 + * Restart raft engine */ public void restartRaftNode() { shutdown(); @@ -471,7 +471,7 @@ public void restartRaftNode() { } /** - * 检查是否活跃,如果不活跃,则重启 + * Check if it is active, if not, restart it. */ public void checkActivity() { Utils.runInThread(() -> { @@ -484,7 +484,7 @@ public void checkActivity() { } /** - * raft peer被销毁,删除日志和数据 + * raft peer is destroyed, deleting logs and data */ public void destroy() { shutdown(); @@ -513,7 +513,7 @@ public void addStateListener(PartitionStateListener listener) { } /** - * 返回所有活跃的peer + * Return all active peers * * @return */ @@ -534,7 +534,7 @@ public Node getRaftNode() { } /** - * 等待Leader被选举 + * Waiting for Leader to be elected * * @param timeOut * @return @@ -563,7 +563,7 @@ public Endpoint waitForLeader(long timeOut) { if (partitionManager.isLocalPartition(this.options.getGroupId())) { log.error("Raft {} leader not found, try to repair!", this.options.getGroupId()); - // TODO 判断raft是否本机,如果是,尝试修复Leader,包括检查配置是否正确 + // TODO Check if raft is local, if so, try to fix the Leader, including checking if the configuration is correct. storeEngine.createPartitionGroups( partitionManager.getPartitionList(getGroupId()).get(0)); } @@ -603,7 +603,7 @@ public String toString() { @Override public void onLeaderStart(long newTerm) { log.info("Raft {} onLeaderStart newTerm is {}", getGroupId(), newTerm); - // 更新shard group对象 + // Update shard group object shardGroup.changeLeader(partitionManager.getStore().getId()); onConfigurationCommitted(getCurrentConf()); @@ -630,9 +630,9 @@ public void onStartFollowing(final PeerId newLeaderId, final long newTerm) { public void onConfigurationCommitted(Configuration conf) { try { - //更新shardlist + //Update shardlist log.info("Raft {} onConfigurationCommitted, conf is {}", getGroupId(), conf.toString()); - // 根据raft endpoint查找storeId + // According to raft endpoint find storeId List peerIds = new ArrayList<>(); for (String peer : RaftUtils.getPeerEndpoints(conf)) { Store store = getStoreByEndpoint(peer); @@ -715,7 +715,7 @@ public TaskManager getTaskManager() { } /** - * 收到PD发送的leader转移指令 + * Received PD's leader transfer command * * @param graphName * @param shard @@ -730,11 +730,11 @@ public Status transferLeader(String graphName, Metapb.Shard shard) { } /** - * 接收到pd发来的修改副本指令 - * 1. 比较新旧peer,查找出新增和删除的peer - * 2. 对于新增的peer,以learner方式加入 - * 3. 监听快照同步事件 - * 4. 快照同步完成后,调用changePeers,修改learner为follower,删除旧的peer + * Received the modification copy instruction sent by pd + * 1. Compare new and old peers, identify added and removed peers. + * 2. For new peers, join as a learner. + * 3. Listen for snapshot synchronization events + * 4. After the snapshot synchronization is completed, call changePeers, change the learner to follower, and delete the old peer. */ public void doChangeShard(final MetaTask.Task task, Closure done) { if (!isLeader()) { @@ -742,22 +742,22 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { } log.info("Raft {} doChangeShard task is {}", getGroupId(), task); - // 如果相同的分区有相同的任务在执行,忽略任务执行 + // If the same partition has the same task executing, ignore task execution. if (taskManager.partitionTaskRepeat(task.getPartition().getId(), task.getPartition().getGraphName(), task.getType().name())) { log.error("Raft {} doChangeShard task repeat, type:{}", getGroupId(), task.getType()); return; } - // 任务未完成,重复执行 + // Task not completed, repeat execution. if (task.getState().getNumber() < MetaTask.TaskState.Task_Stop_VALUE && isLeader()) { Utils.runInThread(() -> { try { - // 不能在状态机中changePeers + // cannot changePeers in the state machine List peers = partitionManager.shards2Peers(task.getChangeShard().getShardList()); HashSet hashSet = new HashSet<>(peers); - // 任务中有相同的peers,说明任务本身有错误,任务忽略 + // Task has the same peers, indicating there is an error in the task itself, task ignored if (peers.size() != hashSet.size()) { log.info("Raft {} doChangeShard peer is repeat, peers:{}", getGroupId(), peers); @@ -772,7 +772,7 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { if (result.getCode() != HgRaftError.TASK_CONTINUE.getNumber()) { log.info("Raft {} doChangeShard is finished, status is {}", getGroupId(), result); - // 任务完成,同步任务状态 + // Task completed, synchronize task status MetaTask.Task newTask; if (result.isOk()) { newTask = task.toBuilder().setState(MetaTask.TaskState.Task_Success) @@ -782,7 +782,7 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { "Raft {} doChangeShard is failure, need to retry, status is {}", getGroupId(), result); try { - // 减少发送次数 + // Reduce send times Thread.sleep(1000); } catch (Exception e) { log.error("wait 1s to resend retry task. got error:{}", @@ -792,7 +792,7 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { .build(); } try { - // 等待过程中,可能已经shut down了 + // During the waiting process, it may have already shut down. if (isLeader()) { storeEngine.addRaftTask(newTask.getPartition().getGraphName(), newTask.getPartition().getId(), @@ -815,7 +815,7 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { task.getPartition().getGraphName(), task.getPartition().getId(), e); } - // db 可能被销毁了,就不要更新了 + // db might have been destroyed, do not update anymore if (this.started) { taskManager.updateTask(newTask); } @@ -829,7 +829,7 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { } }); } else { - // 返回消息是否被处理 + // Whether the message has been processed if (done != null) { done.run(Status.OK()); } @@ -837,11 +837,11 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { } /** - * 收到PD发送的分区之间转移数据 - * 1. 通知目标机器,创建raft - * 2. 从源机器拷贝数据到目标机器 - * 3. 迁移成功后,通知PD修改分区信息 - * 4. 删除源分区 + * Received data transfer between partitions sent by PD + * 1. Notify the target machine to create raft + * 2. Copy data from the source machine to the target machine + * 3. After the migration is successful, notify PD to update partition information. + * 4. Delete source partition * * @return */ @@ -877,17 +877,17 @@ public Status moveData(MetaTask.Task task) { task.getType().name(), status); if (status.isOk()) { - // 向PD汇报任务执行结果 + // Report task execution results to PD partitionManager.reportTask( task.toBuilder().setState(MetaTask.TaskState.Task_Success).build()); - // 更新本地任务状态 + // Update local task status taskManager.updateTask( task.toBuilder().setState(MetaTask.TaskState.Task_Success).build()); } else { partitionManager.reportTask(task.toBuilder() .setState(MetaTask.TaskState.Task_Failure) .setMessage(status.getErrorMsg()).build()); - // 更新本地任务状态 + // Update local task status taskManager.updateTask(task.toBuilder() .setState(MetaTask.TaskState.Task_Failure) .setMessage(status.getErrorMsg()).build()); @@ -897,7 +897,7 @@ public Status moveData(MetaTask.Task task) { } /** - * 对应分区分裂的任务 + * Corresponding to the divisional splitting task * * @param task split partition task * @return task execution result @@ -915,14 +915,14 @@ private Status handleSplitTask(MetaTask.Task task) { for (int i = 0; i < newPartitions.size(); i++) { storeEngine.createPartitionGroups(new Partition(newPartitions.get(i))); } - // 从源机器拷贝数据到目标机器 + // Copy data from the source machine to the target machine status = storeEngine.getDataMover().moveData(task.getPartition(), newPartitions); if (status.isOk()) { var source = Metapb.Partition.newBuilder(targets.get(0)) .setState(Metapb.PartitionState.PState_Normal) .build(); - // 更新本地key range,并同步follower + // Update local key range, and synchronize follower partitionManager.updatePartition(source, true); storeEngine.getDataMover().updatePartitionRange(source, (int) source.getStartKey(), @@ -942,7 +942,7 @@ private Status handleSplitTask(MetaTask.Task task) { } /** - * 对应分区数据移动的task + * Corresponding to partition data movement task * * @param task move partition task * @return task execution result @@ -965,21 +965,21 @@ private Status handleMoveTask(MetaTask.Task task) { } /** - * 对于整个图删除的清空,删除分区, 如果没有其他的图,销毁raft group. - * 需要放到调用move data 之后 + * For the entire graph deletion, clear the deletion partition, if there are no other graphs, destroy the raft group. + * Need to be placed after the call to move data * * @param graphName graph name * @param partitionId partition id - * @param keyStart key start 用于验证 - * @param keyEnd key end 用于验证 - * @param isLeader 是否leader,避免leader漂移,采取move data时候的leader状态 + * @param keyStart key start used for verification + * @param keyEnd key end used for verification + * @param isLeader Whether leader, to avoid leader drifting, the leader status when moving data */ private synchronized void destroyPartitionIfGraphsNull(String graphName, int partitionId, long keyStart, long keyEnd, boolean isLeader) { Partition partition = partitionManager.getPartition(graphName, partitionId); - // key range 校验 + // key range validation if (partition != null && partition.getEndKey() == keyEnd && partition.getStartKey() == keyStart) { log.info("remove partition id :{}, graph:{}", partition.getId(), @@ -987,7 +987,7 @@ private synchronized void destroyPartitionIfGraphsNull(String graphName, int par storeEngine.deletePartition(partitionId, graphName); } - // 没有partition engine的情况 + // without partition engine的情况 if (isLeader && partition == null) { partitionManager.deletePartition(graphName, partitionId); } @@ -1009,10 +1009,10 @@ private synchronized void destroyPartitionIfGraphsNull(String graphName, int par public void snapshot() { log.info("Raft {} send snapshot command. ", this.getGroupId()); - // 空指令,占位 + // Null instruction, placeholder this.addRaftTask( RaftOperation.create(RaftOperation.BLANK_TASK), status -> { - // 生成快照指令 + // Generate snapshot command this.addRaftTask( RaftOperation.create(RaftOperation.DO_SNAPSHOT), status2 -> { }); @@ -1037,14 +1037,14 @@ protected void doSnapshot(Closure done) { } public void addBlankRaftTask() { - // 空指令,占位 + // Null instruction, placeholder this.addRaftTask( RaftOperation.create(RaftOperation.BLANK_TASK), status -> { }); } private void handleCleanOp(CleanDataRequest request) { - // 避免清理数据过程中的leader漂移 + // Avoid leader drift during data cleanup process boolean isLeader = isLeader(); var partition = partitionManager.getPartition(request.getGraphName(), request.getPartitionId()); @@ -1156,8 +1156,8 @@ public void onDestroyed(PeerId peer) { } /** - * 监听replicator状态改变,判断快照是否同步完成 - * 检查是否存在changeShard任务,如果存在,调用changeShard + * Listen for changes in replicator status to determine if the snapshot is fully synchronized. + * Check if there is a changeShard task, if it exists, call changeShard. */ @Override public void stateChanged(final PeerId peer, final ReplicatorState newState) { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionInstructionProcessor.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionInstructionProcessor.java index a9140701a4..65830b7ba8 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionInstructionProcessor.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionInstructionProcessor.java @@ -51,7 +51,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; /** - * PD发给Store的分区指令处理器 + * PD sends partition instruction processor to Store */ public class PartitionInstructionProcessor implements PartitionInstructionListener { @@ -78,7 +78,7 @@ public void onChangeShard(long taskId, Partition partition, ChangeShard changeSh PartitionEngine engine = storeEngine.getPartitionEngine(partition.getId()); if (engine != null) { - // 清理所有的任务,有失败的情况 + // Clean up all tasks, with failures occurring. engine.getTaskManager() .deleteTask(partition.getId(), MetaTask.TaskType.Change_Shard.name()); } @@ -131,8 +131,8 @@ public void onTransferLeader(long taskId, Partition partition, TransferLeader tr } /** - * Leader接收到PD发送的分区分裂任务 - * 添加到raft任务队列,由raft进行任务分发。 + * Leader receives the partition splitting task sent by PD. + * Added to the raft task queue, task distribution is handled by raft. */ @Override public void onSplitPartition(long taskId, Partition partition, SplitPartition splitPartition, @@ -144,7 +144,7 @@ public void onSplitPartition(long taskId, Partition partition, SplitPartition sp } if (engine != null && engine.isLeader()) { - // 先应答,避免超时造成pd重复发送 + // Respond first to avoid pd retransmission due to timeout. consumer.accept(0); String graphName = partition.getGraphName(); @@ -168,8 +168,8 @@ public void onSplitPartition(long taskId, Partition partition, SplitPartition sp } /** - * Leader接收到PD发送的rocksdb compaction任务 - * 添加到raft任务队列,由raft进行任务分发。 + * Leader receives the rocksdb compaction task sent by PD + * Added to the raft task queue, task distribution is handled by raft. */ @Override public void onDbCompaction(long taskId, Partition partition, DbCompaction dbCompaction, @@ -210,7 +210,7 @@ public void onMovePartition(long taskId, Partition partition, MovePartition move } if (engine != null && engine.isLeader()) { - // 先应答,避免超时造成pd重复发送 + // Respond first to avoid pd retransmission due to timeout. consumer.accept(0); String graphName = partition.getGraphName(); @@ -278,7 +278,7 @@ public void onPartitionKeyRangeChanged(long taskId, Partition partition, partitionManager.getPartition(partition.getGraphName(), partition.getId()); if (localPartition == null) { - // 如果分区数据为空,本地不会存储 + // If the partition data is empty, it will not be stored locally. localPartition = partitionManager.getPartitionFromPD(partition.getGraphName(), partition.getId()); LOG.info("onPartitionKeyRangeChanged, get from pd:{}-{} -> {}", diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionStateListener.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionStateListener.java index 8fd5e129be..ad73f95e8a 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionStateListener.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionStateListener.java @@ -25,10 +25,10 @@ public interface PartitionStateListener { - // 分区角色发生改变 + // Partition role changed void partitionRoleChanged(Partition partition, PartitionRole newRole); - // 分区发生改变 + // Partition has changed void partitionShardChanged(Partition partition, List oldShards, List newShards); } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java index 5ef441659c..06175fb7a2 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java @@ -108,23 +108,23 @@ void loadSnapshot(String snapshotPath, String graph, int partId, long version) t long getLatestSequenceNumber(String graph, int partId); - // 扫描分区从 seqnum 开始的 kv + // Scan partition starting from seqnum for kv ScanIterator scanRaw(String graph, int partId, long seqNum) throws HgStoreException; void ingestSstFile(String graph, int partId, Map> sstFiles) throws HgStoreException; - //提交分区分裂,删除旧数据 - // 删除分区数据 + //Submit partition split, delete old data + // Delete partition data boolean deletePartition(String graph, int partId); - //清理分区,删除多余的数据 + //Clean up partitions, delete unnecessary data boolean cleanPartition(String graph, int partId); boolean cleanPartition(String graph, int partId, long startKey, long endKey, CleanType cleanType); - //所有指定分区图的所有 table 名 + // All table names for all specified partition diagrams List getTableNames(String graph, int partId); TxBuilder txBuilder(String graph, int partId); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java index 779a86bacc..d741cd1f19 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java @@ -119,7 +119,7 @@ public void onDBSessionReleased(RocksDBSession dbSession) { public static HugeConfig initRocksdb(Map rocksdbConfig, RocksdbChangedListener listener) { - // 注册 rocksdb 配置 + // Register rocksdb configuration OptionSpace.register("rocksdb", "org.apache.hugegraph.rocksdb.access.RocksDBOptions"); RocksDBOptions.instance(); HugeConfig hConfig = new HugeConfig(new MapConfiguration(rocksdbConfig)); @@ -136,7 +136,7 @@ public static String getDbName(int partId) { dbName = String.format("%05d", partId); dbNames.put(partId, dbName); } - // 每个分区对应一个 rocksdb 实例,因此 rocksdb 实例名为 partId + // Each partition corresponds to a rocksdb instance, so the rocksdb instance name is partId. return dbName; } @@ -232,12 +232,12 @@ public ScanIterator scan(String graph, int code, String table, byte[] start, byt } /** - * 根据 keyCode 范围返回数据,左闭右开 + * According to keyCode range return data, left closed right open. * * @param graph * @param table - * @param codeFrom 起始 code,包含该值 - * @param codeTo 结束 code,不包含该值 + * @param codeFrom Starting code, including this value + * @param codeTo End code, does not include this value * @return * @throws HgStoreException */ @@ -435,15 +435,15 @@ public void batchGet(String graph, String table, Supplier= startKey && code < endKey; return (cleanType == CleanType.CLEAN_TYPE_KEEP_RANGE) == flag; }); - // 可能被 destroy 了 + // May have been destroyed. if (HgStoreEngine.getInstance().getPartitionEngine(partId) != null) { taskManager.updateAsyncTaskState(partId, graph, cleanTask.getId(), AsyncTaskState.SUCCESS); @@ -573,8 +573,8 @@ public boolean cleanPartition(String graph, int partId, long startKey, long endK } /** - * 清理分区数据,删除非本分区的数据 - * 遍历 partId 的所有 key,读取 code,if code >= splitKey 生成新的 key,写入 newPartId + * Clean up partition data, delete data not belonging to this partition. + * Traverse all keys of partId, read code, if code >= splitKey generate a new key, write to newPartId */ private boolean cleanPartition(Partition partition, Function belongsFunction) { @@ -602,7 +602,7 @@ private boolean cleanPartition(Partition partition, if (counter == 0) { op.prepare(); } - op.delete(table, col.name); // 删除旧数据 + op.delete(table, col.name); // delete old data if (++counter > batchSize) { op.commit(); counter = 0; @@ -667,11 +667,11 @@ private RocksDBSession getSession(String graphName, int partId) throws HgStoreEx } /** - * 获取 dbsession,不更新 dbsession 活跃时间 + * Get dbsession, do not update dbsession active time */ @Override public RocksDBSession getSession(int partId) throws HgStoreException { - // 每个分区对应一个 rocksdb 实例,因此 rocksdb 实例名为 rocksdb + partId + // Each partition corresponds to a rocksdb instance, so the rocksdb instance name is rocksdb + partId String dbName = getDbName(partId); RocksDBSession dbSession = factory.queryGraphDB(dbName); if (dbSession == null) { @@ -685,7 +685,7 @@ public RocksDBSession getSession(int partId) throws HgStoreException { "failed to create a new graph db: {}", dbName); } } - dbSession.setDisableWAL(true); //raft 模式,关闭 rocksdb 日志 + dbSession.setDisableWAL(true); // raft mode, disable rocksdb log return dbSession; } @@ -719,7 +719,7 @@ public void createTable(String graph, int partId, String table) { @Override public void deleteTable(String graph, int partId, String table) { dropTable(graph, partId, table); - // todo 检查表是否为空,为空则真实删除表 + // todo Check if the table is empty, if empty then truly delete the table // try (RocksDBSession session = getOrCreateGraphDB(graph, partId)) { // session.deleteTables(table); // } @@ -735,7 +735,7 @@ public void dropTable(String graph, int partId, String table) { } /** - * 对 rocksdb 进行 compaction + * Perform compaction on RocksDB */ @Override public boolean dbCompaction(String graphName, int partitionId) { @@ -743,7 +743,7 @@ public boolean dbCompaction(String graphName, int partitionId) { } /** - * 对 rocksdb 进行 compaction + * Perform compaction on RocksDB */ @Override public boolean dbCompaction(String graphName, int partitionId, String tableName) { @@ -761,14 +761,14 @@ public boolean dbCompaction(String graphName, int partitionId, String tableName) } /** - * 销毁图,并删除数据文件 + * Destroy the map, and delete the data file. * * @param graphName * @param partId */ @Override public void destroyGraphDB(String graphName, int partId) throws HgStoreException { - // 每个图每个分区对应一个 rocksdb 实例,因此 rocksdb 实例名为 rocksdb + partId + // Each graph each partition corresponds to a rocksdb instance, so the rocksdb instance name is rocksdb + partId String dbName = getDbName(partId); factory.destroyGraphDB(dbName); @@ -904,7 +904,7 @@ public Tx build() { return new Tx() { @Override public void commit() throws HgStoreException { - op.commit(); // commit发生异常后,必须调用rollback,否则造成锁未释放 + op.commit(); // After an exception occurs in commit, rollback must be called, otherwise it will cause the lock not to be released. dbSession.close(); } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DataMover.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DataMover.java index 51e2ae06b0..a348f561c7 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DataMover.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DataMover.java @@ -28,7 +28,7 @@ import com.alipay.sofa.jraft.Status; /** - * 数据转移接口,实现分区分裂和合并,支持跨机器转移数据 + * Data transfer interface, implementing partition splitting and merging, supporting cross-machine data transfer. */ public interface DataMover { @@ -37,8 +37,8 @@ public interface DataMover { void setCmdClient(HgCmdClient client); /** - * 拷贝分区source内的数据到其他分区targets - * 一个分区,迁移到多个分区 + * Copy the data within the partition source to other partitions targets + * One partition, migrating to multiple partitions * * @param source source partition * @param targets target partitions @@ -48,8 +48,8 @@ public interface DataMover { Status moveData(Metapb.Partition source, List targets) throws Exception; /** - * 将source target的数据全部拷贝到target上 - * 从一个分区迁移到另外一个分区 + * Copy all data from source to target. + * Migrate from one partition to another partition * * @param source source partition * @param target target partition @@ -58,18 +58,18 @@ public interface DataMover { */ Status moveData(Metapb.Partition source, Metapb.Partition target) throws Exception; - // 同步副本之间的分区状态 + // Synchronize the partition state between replicas UpdatePartitionResponse updatePartitionState(Metapb.Partition partition, Metapb.PartitionState state); - // 同步副本之间分区的范围 + // Synchronization of the range of partitions between replicas UpdatePartitionResponse updatePartitionRange(Metapb.Partition partition, int startKey, int endKey); - // 清理分区partition内的无效数据 + // Clean up invalid data within the partitionpartition void cleanData(Metapb.Partition partition); - // 写入数据 + // Write data void doWriteData(BatchPutRequest request); void doCleanData(CleanDataRequest request); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DefaultDataMover.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DefaultDataMover.java index a2a730e3ca..aeca3a3cae 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DefaultDataMover.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/DefaultDataMover.java @@ -71,13 +71,13 @@ public void setCmdClient(HgCmdClient client) { public Status moveData(Metapb.Partition source, List targets) throws Exception { Status status = Status.OK(); - // 开始移动数据之前,先把分区下线 + // Before starting to move data, take the partition offline first. UpdatePartitionResponse response = updatePartitionState(source, Metapb.PartitionState.PState_Offline); if (response.getStatus().isOK()) { status = moveData(source, targets, DefaultDataMover::findPartition); - // 数据迁移成功后,设置新分区范围和上线新分区 + // Data migration successful, set new partition range and launch new partition. for (var target : targets) { if (status.isOk()) { if (!(updatePartitionRange(target, (int) target.getStartKey(), @@ -101,7 +101,7 @@ && updatePartitionState(target, @Override public Status moveData(Metapb.Partition source, Metapb.Partition target) throws Exception { - // 只写入 target + // only write to target return moveData(source, Collections.singletonList(target), (partitions, integer) -> target); } @@ -178,7 +178,7 @@ private Status moveData(Metapb.Partition source, List targets, @Override public UpdatePartitionResponse updatePartitionState(Metapb.Partition partition, Metapb.PartitionState state) { - // 分区分裂时,主动需要查找 leader 进行同步信息 + // When the partition splits, it actively needs to find the leader to synchronize information. UpdatePartitionRequest request = new UpdatePartitionRequest(); request.setWorkState(state); request.setPartitionId(partition.getId()); @@ -189,7 +189,7 @@ public UpdatePartitionResponse updatePartitionState(Metapb.Partition partition, @Override public UpdatePartitionResponse updatePartitionRange(Metapb.Partition partition, int startKey, int endKey) { - // 分区分裂时,主动需要查找 leader 进行同步信息 + // When the partition splits, it actively needs to find the leader for information synchronization. UpdatePartitionRequest request = new UpdatePartitionRequest(); request.setStartKey(startKey); request.setEndKey(endKey); @@ -228,7 +228,7 @@ public void doWriteData(BatchPutRequest request) { @Override public void doCleanData(CleanDataRequest request) { - // raft 执行真实数据的清理 + // raft performs real data cleanup businessHandler.cleanPartition(request.getGraphName(), request.getPartitionId(), request.getKeyStart(), request.getKeyEnd(), request.getCleanType()); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/FilterIterator.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/FilterIterator.java index a70abcd629..e3c1380b93 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/FilterIterator.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/FilterIterator.java @@ -72,7 +72,7 @@ public boolean hasNext() { current.name); entry.columns(Arrays.asList(columns)); } else { - // 有可能存在包含多个 column 的情况 + // There may be cases that contain multiple columns entry.columns(Arrays.asList(columns)); continue; } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyCreator.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyCreator.java index ca14486d95..072d09cc4a 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyCreator.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyCreator.java @@ -62,7 +62,7 @@ public void clearCache(Integer partId) { } /** - * 从key中解析出keyCode + * Parse keyCode from key */ public int parseKeyCode(byte[] innerKey) { return Bits.getShort(innerKey, innerKey.length - Short.BYTES); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyFilter.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyFilter.java index 9338839013..34dc46063b 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyFilter.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/InnerKeyFilter.java @@ -27,7 +27,7 @@ public class InnerKeyFilter implements ScanIterator { final int codeFrom; final int codeTo; - //是否进行code过滤,启动该选项,返回key的尾部包含code + // Whether to perform code filtering, enable this option, return the key's tail containing code final boolean codeFilter; ScanIterator iterator; T current = null; @@ -80,11 +80,11 @@ public boolean isValid() { public T next() { T column = current; if (!codeFilter) - // 去掉图ID和hash后缀 + // Remove the image ID and hash suffix { column.name = Arrays.copyOfRange(column.name, Short.BYTES, column.name.length - Short.BYTES); - } else// 去掉图ID + } else // Remove graph ID { column.name = Arrays.copyOfRange(column.name, Short.BYTES, column.name.length); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdClient.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdClient.java index 3bbb2f9da8..6a73639e67 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdClient.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdClient.java @@ -108,7 +108,7 @@ public Store getStoreInfo(final String address) { } /** - * 批量插入数据 + * Batch insert data * * @param request * @return @@ -118,7 +118,7 @@ public BatchPutResponse batchPut(BatchPutRequest request) { } /** - * 清理无效数据 + * Clean up invalid data * * @param request * @return @@ -128,7 +128,7 @@ public CleanDataResponse cleanData(CleanDataRequest request) { } /** - * 通过raft更新本地分区信息 + * Through raft to update local partition information * * @param request * @return @@ -138,7 +138,7 @@ public UpdatePartitionResponse raftUpdatePartition(UpdatePartitionRequest reques } /** - * 查找Leader,错误重试,处理Leader重定向 + * Find Leader, retry on error, handle Leader redirection * * @param request * @return @@ -164,7 +164,7 @@ public HgCmdBase.BaseResponse tryInternalCallSyncWithRpc(HgCmdBase.BaseRequest r } else if (HgCmdProcessor.Status.LEADER_REDIRECT == response.getStatus() && response.partitionLeaders != null ) { - // 当返回leader 漂移,并且partitionLeaders 不为空时,需要重新设置leader + // When returning leader drift, and partitionLeaders is not empty, need to reset the leader. } else { log.error( "HgCmdClient tryInternalCallSyncWithRpc error msg {} leaders is {}", diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdProcessor.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdProcessor.java index 039184ca6d..e0710ef97e 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdProcessor.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/cmd/HgCmdProcessor.java @@ -34,7 +34,7 @@ import lombok.extern.slf4j.Slf4j; /** - * 快照同步rpc处理器,leader批量入库完成后,基于seqnum读取新增的kv,批量发送给follower. + * Snapshot synchronization rpc processor, after the leader completes batch storage, reads the newly added kv based on seqnum and sends it in batches to the follower. * * @param */ @@ -139,7 +139,7 @@ public void handleDestroyRaft(DestroyRaftRequest request, DestroyRaftResponse re } /** - * raft 通知副本同步执行 + * raft notify replica synchronization execution * * @param request * @param response diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphIdManager.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphIdManager.java index 5cdffc9a37..c98b03935d 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphIdManager.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphIdManager.java @@ -29,7 +29,7 @@ import com.google.protobuf.Int64Value; /** - * GraphId管理器,维护一个自增循环ID,负责管理GraphName和GraphId的映射 + * GraphId Manager, maintains a self-incrementing circular ID, responsible for managing the mapping between GraphName and GraphId. */ public class GraphIdManager extends PartitionMetaStore { @@ -69,7 +69,7 @@ public GraphIdManager(DBSessionBuilder sessionBuilder, int partitionId) { } /** - * 获取一个图的id + * Get the id of a graph */ public long getGraphId(String graphName) { Long l = graphIdCache.get(graphName); @@ -97,7 +97,7 @@ public long getGraphId(String graphName) { } /** - * 释放一个图id + * Release a graph id */ public long releaseGraphId(String graphName) { long gid = getGraphId(graphName); @@ -112,10 +112,10 @@ public long releaseGraphId(String graphName) { } /** - * 获取自增循环不重复id, 达到上限后从0开始自增 + * Get auto-increment non-repetitive id, start from 0 after reaching the limit. * * @param key key - * @param max id上限,达到该值后,重新从0开始自增 + * @param max max id limit, after reaching this value, it will reset to 0 and start incrementing again. * @return id */ protected long getCId(String key, long max) { @@ -124,7 +124,7 @@ protected long getCId(String key, long max) { Int64Value value = get(Int64Value.parser(), cidNextKey); long current = value != null ? value.getValue() : 0L; long last = current == 0 ? max - 1 : current - 1; - // 查找一个未使用的cid + // Find an unused cid List ids = scan(Int64Value.parser(), genCIDSlotKey(key, current), genCIDSlotKey(key, max)); for (Int64Value id : ids) { @@ -151,16 +151,16 @@ protected long getCId(String key, long max) { if (current == last) { return -1; } - // 保存当前id,标记已被使用 + // Save current id, mark as used put(genCIDSlotKey(key, current), Int64Value.of(current)); - // 保存下一次遍历的id + // Save the id for the next traversal put(cidNextKey, Int64Value.of(current + 1)); return current; } } /** - * 返回已使用Cid的key + * Return key with used Cid */ private byte[] genCIDSlotKey(String key, long value) { byte[] keySlot = MetadataKeyHelper.getCidSlotKeyPrefix(key); @@ -171,7 +171,7 @@ private byte[] genCIDSlotKey(String key, long value) { } /** - * 删除一个循环id,释放id值 + * Delete a loop ID, release the ID value */ protected void delCId(String key, long cid) { delete(genCIDSlotKey(key, cid)); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphManager.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphManager.java index f579fcdc39..26f157fd99 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphManager.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/GraphManager.java @@ -40,8 +40,8 @@ public GraphManager(MetadataOptions options, PdProvider pdProvider) { } /** - * 修改图 - * 此处不加锁,要求graph是被克隆的,进制修改原始对象 + * Modify image + * This place does not add a lock, requiring the graph to be cloned, forbidden to modify the original object. * * @param graph * @return diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/MetadataKeyHelper.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/MetadataKeyHelper.java index b657920d6a..8a58b1322a 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/MetadataKeyHelper.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/MetadataKeyHelper.java @@ -61,7 +61,7 @@ public static byte[] getPartitionPrefixKey(String graph) { } /** - * 查询分区内的所有partition prefix, 不包含 graph name + * Query all partition prefixes within the partition, not including the graph name. * * @return */ diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Partition.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Partition.java index 665f981aa2..05196abbb7 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Partition.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Partition.java @@ -32,7 +32,7 @@ public class Partition implements Cloneable { private long startKey; private long endKey; private long version; - // shardlist版本,shardlist每次改变加1 + // shardlist version, shardlist increments by 1 each time it changes // private long confVer; private Metapb.PartitionState workState; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java index 517de1beb2..c71ae27213 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java @@ -52,7 +52,7 @@ import lombok.extern.slf4j.Slf4j; /** - * Partition对象管理策略,每次修改需要克隆一份,并且版本号递增 + * Partition object management strategy, each modification requires cloning a copy, and the version number is incremented. */ @Slf4j public class PartitionManager extends GlobalMetaStore { @@ -65,12 +65,12 @@ public class PartitionManager extends GlobalMetaStore { private final boolean useRaft; private final HgStoreEngineOptions options; private final List partitionChangedListeners; - // 读写锁对象 + // Read-write lock object private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); private final PartitionMetaStoreWrapper wrapper = new PartitionMetaStoreWrapper(); - // 记录本机所有的分区信息,与rocksdb存储保持一致 + // Record all partition information of this machine, consistent with rocksdb storage. private Map> partitions; public PartitionManager(PdProvider pdProvider, HgStoreEngineOptions options) { @@ -116,17 +116,17 @@ public void addPartitionChangedListener(PartitionChangedListener listener) { } /** - * 判断存储路径为分区id或者分区id_开头 + * Judge the storage path as either the partition id or starting with partition id_ * * @param detections dir list * @param partitionId partition id - * @param checkLogDir : 是否包含子目录 log ( raft snapshot 和log 分离,需要进一步检查) + * @param checkLogDir : whether it includes the subdirectory log (raft snapshot and log separation, further checks are needed) * @return true if contains partition id, otherwise false */ private Boolean checkPathContains(File[] detections, int partitionId, boolean checkLogDir) { String partitionDirectory = String.format("%05d", partitionId); for (int x = 0; x < detections.length; x++) { - // 一定是以分区id命名的文件夹下 + // Must be in a folder named after the partition id if (detections[x].isDirectory()) { String tmp = detections[x].getName(); if (tmp.equals(partitionDirectory) || tmp.startsWith(partitionDirectory + "_")) { @@ -145,9 +145,9 @@ private Boolean checkPathContains(File[] detections, int partitionId, boolean ch } /** - * 根据配置文件根目录,循环查找分区的存储路径 - * 根据约定db数据在dataPath/db/分区id目录,raft数据在dataPath/raft/分区id目录 - * 检测分区存储文件夹是否存在 + * According to the root directory of the profile, loop through to find the storage path of the partition. + * According to the agreement, db data is in the dataPath/db/partition_id directory, and raft data is in the dataPath/raft/partition_id directory. + * Check if the partition storage folder exists */ private Boolean resetPartitionPath(int partitionId) { List dataPaths = Arrays.asList(this.options.getDataPath().split(",")); @@ -156,7 +156,7 @@ private Boolean resetPartitionPath(int partitionId) { boolean isDataOk = false; boolean isRaftOk = false; - // 检查 db 目录 + // Check db directory for (int i = 0; i < dataPaths.size(); i++) { String dbPath = Paths.get(dataPaths.get(i), HgStoreEngineOptions.DB_Path_Prefix).toAbsolutePath() @@ -180,7 +180,7 @@ private Boolean resetPartitionPath(int partitionId) { } } - // 检查 raft目录 + // Check raft directory for (int i = 0; i < raftPaths.size(); i++) { String raftPath = Paths.get(raftPaths.get(i), HgStoreEngineOptions.Raft_Path_Prefix).toAbsolutePath() @@ -192,7 +192,7 @@ private Boolean resetPartitionPath(int partitionId) { File[] raftFiles = raftFile.listFiles(); if (this.checkPathContains(raftFiles, partitionId, true)) { Metapb.PartitionRaft location = storeMetadata.getPartitionRaft(partitionId); - // 兼容版本升级 + // Compatible version upgrade if (location == null || !Objects.equals(location.getRaftLocation(), raftPaths.get(i))) { Metapb.PartitionRaft newLocation = Metapb.PartitionRaft.newBuilder() @@ -214,14 +214,45 @@ private Boolean resetPartitionPath(int partitionId) { } /** - * 从本地storage中读取分区 + * Read the partition from local storage */ private void loadPartitions() { byte[] key = MetadataKeyHelper.getPartitionPrefixKey(); long storeId = getStore().getId(); - // 从data path中读取 partition - // 记录有哪些分区 + // Read partition from data path + // Record which partitions + +```python +// 显示所有分区信息 +diskutil list + +// 创建一个新分区 +diskutil partitionDisk disk0 1 GPT HFS+ NewPartition 100% + +// 格式化分区 +diskutil eraseVolume HFS+ NewPartition disk0s3 + +// 列出分区信息 +diskutil info disk0s3 +``` + +```python +// Record which partitions + +```python +// Display all partition information +diskutil list + +// Create a new partition +diskutil partitionDisk disk0 1 GPT HFS+ NewPartition 100% + +// Format the partition +diskutil eraseVolume HFS+ NewPartition disk0s3 + +// List partition information +diskutil info disk0s3 +``` var partIds = new HashSet(); for (String path : this.options.getDataPath().split(",")) { File[] dirs = new File(path + "/" + HgStoreEngineOptions.DB_Path_Prefix).listFiles(); @@ -241,7 +272,7 @@ private void loadPartitions() { } } - // 一次按照分区读取 + // Once according to the partition read for (int partId : partIds) { if (!resetPartitionPath(partId)) { log.error("partition " + partId + " Directory not exists,options " + @@ -257,7 +288,7 @@ private void loadPartitions() { var shards = pdProvider.getShardGroup(metaPart.getId()).getShardsList(); if (pdPartition != null) { - // 判断是否包含本store id + // Check if it contains this store id if (shards.stream().anyMatch(s -> s.getStoreId() == storeId)) { isLegeal = true; } @@ -269,12 +300,12 @@ private void loadPartitions() { } Partition partition = new Partition(metaPart); - partition.setWorkState(Metapb.PartitionState.PState_Normal); // 启动恢复工作状态 + partition.setWorkState(Metapb.PartitionState.PState_Normal); // Start recovery work state partitions.get(graph).put(partition.getId(), partition); log.info("load partition : {} -{}", partition.getGraphName(), partition.getId()); } else { - // 无效 + // Invalid // removePartitionFromLocalDb(graph, partId); // var businessHandler = HgStoreEngine.getInstance().getBusinessHandler(); // businessHandler.truncate(graph, partId); @@ -293,8 +324,8 @@ public List loadPartitionsFromDb(int partitionId) { } /** - * 从PD同步分区,并删除本地多余的分区 - * 同步过程中,新增分区需要保存到本地,已有的分区信息与本地进行合并 + * Synchronize from PD and delete the extra local partitions. + * During the synchronization process, new partitions need to be saved locally, and the existing partition information is merged with the local data. */ public void syncPartitionsFromPD(Consumer delCallback) throws PDException { Lock writeLock = readWriteLock.writeLock(); @@ -314,16 +345,16 @@ public void syncPartitionsFromPD(Consumer delCallback) throws PDExcep } }); - // 遍历本地图,删除本地多余,追加新的。 + // Traverse the local map, delete the local extras, and append the new ones. partitions.forEach((graphName, v) -> { Map partitionsFrpd = graphPtFrpd.get(graphName); v.forEach((id, pt) -> { if (partitionsFrpd == null || !partitionsFrpd.containsKey(id)) { - // 本地的分区,pd已不存在,需要删除 + // Local partition, pd no longer exists, needs to be deleted delCallback.accept(pt); removePartition(pt.getGraphName(), pt.getId()); } else { - // 修改shard信息 + // Modify shard information // Partition ptFrpd = partitionsFrpd.get(id); // pt.setShardsList(ptFrpd.getShardsList()); savePartition(pt, true, true); @@ -333,7 +364,7 @@ public void syncPartitionsFromPD(Consumer delCallback) throws PDExcep if (partitionsFrpd != null) { partitionsFrpd.forEach((id, pt) -> { if (!v.containsKey(id)) { - // 新增的分区 + // New partition added savePartition(pt, true); } }); @@ -377,7 +408,7 @@ public Partition updatePartition(Metapb.Partition partition, boolean updateRange } /** - * 增加partition对象 + * Add partition object * * @param partition * @return @@ -412,7 +443,7 @@ public void updatePartitionRangeOrState(UpdatePartitionRequest req) { } /** - * 强制更新 partition,不校验 version + * Force update partition, do not verify version * * @param partition * @return @@ -429,7 +460,7 @@ public Partition loadPartitionFromSnapshot(Partition partition) { } /** - * 查找属于本机的Partiton,优先从本地查找,本地未找到,询问pd。 + * Find the Partition belonging to this machine, prioritize searching locally, if not found locally, inquire with pd. * * @param graph * @param partId @@ -446,7 +477,7 @@ public Partition findPartition(String graph, Integer partId) { if (partition != null) { if (isLocalPartition(partition)) { - // 属于本机的partion,保存partition + // Belong to the local machine's partition, save partition Lock writeLock = readWriteLock.writeLock(); writeLock.lock(); try { @@ -473,7 +504,7 @@ public int getPartitionIdByCode(String graph, int code) { } /** - * 从pd获取拉取分区信息,并和本地的分区信息进行合并。leader和shardList取自本地 + * Get partition information from pd and merge it with local partition information. Leader and shardList are taken from local. */ public Partition getPartitionFromPD(String graph, int partId) { pdProvider.invalidPartitionCache(graph, partId); @@ -484,7 +515,7 @@ public Partition getPartitionFromPD(String graph, int partId) { if (partitions.containsKey(graph)) { Partition local = partitions.get(graph).get(partId); if (local != null) { - //更新本地的key范围,保证pd和本地分区信息的一致性 + // Update the local key range, ensuring consistency between pd and local partition information local.setStartKey(partition.getStartKey()); local.setEndKey(partition.getEndKey()); savePartition(local, true, true); @@ -498,8 +529,8 @@ public Partition getPartitionFromPD(String graph, int partId) { } /** - * 是否是本地的分区 - * 对于批处理入库,只有leader才属于本地 + * Whether it is a local partition + * For batch processing storage, only the leader is local. * * @param partition * @return @@ -519,8 +550,8 @@ public boolean isLocalPartition(Partition partition) { } /** - * 是否是本地的分区 - * 对于批处理入库,只有leader才属于本地 + * Whether it is a local partition + * For batch inventory storage, only the leader is local. * * @return */ @@ -529,8 +560,8 @@ public boolean isLocalPartition(int partId) { } /** - * 存储partition信息,同步保存到内存和rocksdb - * 不更新key range + * Store partition information, synchronize saving to memory and rocksdb + * Not update key range */ private void savePartition(Partition partition, Boolean changeLeader) { @@ -538,7 +569,7 @@ private void savePartition(Partition partition, Boolean changeLeader) { } /** - * 保存partition 信息 + * Save partition information * * @param partition partition * @param changeLeader is change leader @@ -571,16 +602,16 @@ private void savePartition(Partition partition, Boolean changeLeader, Boolean ch graph.setGraphName(partition.getGraphName()); graphManager.updateGraph(graph); - // 更新PD cache,后序优化,store不依赖pdclient cache + // Update PD cache, subsequent optimization, store does not depend on pdclient cache pdProvider.updatePartitionCache(partition, changeLeader); partitionChangedListeners.forEach(listener -> { - listener.onChanged(partition); // 通知raft,进行同步分区信息同步 + listener.onChanged(partition); // Notify raft, synchronize partition information synchronization }); } /** - * 更新shard group到db, 同时更新shardGroups对象 + * Update shard group to db, while updating the shardGroups object * * @param shardGroup */ @@ -594,8 +625,8 @@ public void updateShardGroup(ShardGroup shardGroup) { } /** - * 查找 partition id对应的shard group。 - * 依次从 raft node/local db/ pd 读取。 + * Find the shard group corresponding to the partition id. + * Read in sequence from raft node/local db/ pd. * * @param partitionId * @return @@ -647,19 +678,19 @@ private void removePartitionFromLocalDb(String graphName, Integer partId) { } /** - * 删除图数据,删除本地数据,并删除PD上的分区信息 + * Delete graph data, delete local data, and delete partition information on PD. */ public Partition deletePartition(String graphName, Integer partId) { removePartition(graphName, partId); return pdProvider.delPartition(graphName, partId); } - // 获取本地Store信息 + // Get local Store information public Store getStore() { return storeMetadata.getStore(); } - // 注册会修改StoreId,需要重置 + // Registration will modify StoreId, need to reset public void setStore(Store store) { Lock writeLock = readWriteLock.writeLock(); writeLock.lock(); @@ -709,7 +740,7 @@ public boolean hasPartition(String graphName, int partitionId) { } /** - * 获取图在本机中所有Leader 分区 + * Get all Leader partitions in this graph locally. * * @param graph * @return @@ -727,7 +758,7 @@ public List getLeaderPartitionIds(String graph) { } /** - * 生成分区peer字符串,包含优先级信息 * + * Generate partition peer string, containing priority information * * * @param shardGroup * @return @@ -759,7 +790,7 @@ public List shards2Peers(List shards) { } /** - * 是否是本地store + * Whether it is a local store * * @param store * @return @@ -773,7 +804,7 @@ public PartitionRole getLocalRoleFromShard(Partition partition) { } /** - * 修改分区角色 + * Modify partition role */ public Partition changeLeader(Partition pt, List shards, long term) { Lock writeLock = readWriteLock.writeLock(); @@ -792,7 +823,7 @@ public Partition changeLeader(Partition pt, List shards, long term } /** - * 根据raft peers清单,重建shardList + * According to the raft peers list, rebuild shardList */ public Partition changeShards(Partition pt, List shards) { Lock writeLock = readWriteLock.writeLock(); @@ -811,16 +842,16 @@ public Partition changeShards(Partition pt, List shards) { } /** - * 拆分partition对象 + * Split partition object */ public List updatePartitionToPD(List partitions) throws PDException { - // 更新本地分区信息,以及cache信息 + // Update local partition information, as well as cache information return pdProvider.updatePartition(partitions); } /** - * 根据raft address查找Store + * According to raft address to find Store */ public Store getStoreByRaftEndpoint(ShardGroup group, String endpoint) { final Store[] result = {new Store()}; @@ -845,7 +876,7 @@ public Shard getShardByRaftEndpoint(ShardGroup group, String endpoint) { } /** - * raft存储路径 + * raft storage path * * @param groupId * @return location/raft/groupId/ @@ -859,7 +890,7 @@ public String getRaftDataPath(int groupId) { } /** - * raft snapshot 的路径,要和 db同一个盘上,便于hard link + * raft snapshot path should be on the same disk as the db, convenient for hard link * * @param groupId raft group id * @return location/snapshot/0000x/ @@ -874,7 +905,7 @@ public String getRaftSnapShotPath(int groupId) { } /** - * db存储路径 + * db storage path * * @return location/db */ @@ -894,7 +925,7 @@ public void reportTask(MetaTask.Task task) { } /** - * 修改partion的state状态 + * Modify the state of the partition state */ public List changePartitionToOnLine(List partitions) { List newPartitions = new ArrayList<>(); @@ -909,7 +940,7 @@ public PartitionMetaStoreWrapper getWrapper() { } /** - * Partition对象被修改消息 + * Partition object is modified message */ public interface PartitionChangedListener { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionStats.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionStats.java index 3163dffed8..bb035390d9 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionStats.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionStats.java @@ -27,15 +27,15 @@ @Data public class PartitionStats { - // 分区leader所在shard + // Partition leader in shard Metapb.Shard leader; - // 分区离线的shard + // Partition offline shard List offlineShards = new ArrayList<>(); long committedIndex; long leaderTerm; long approximateSize; long approximateKeys; - // 分区ID + // Partition ID private int id; private String namespace; private String graphName; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Shard.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Shard.java index 20f511ea85..3e3d10f038 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Shard.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Shard.java @@ -22,7 +22,7 @@ import lombok.Data; /** - * 一个分片 + * A shard */ @Data public class Shard { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/ShardGroup.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/ShardGroup.java index ad16a402cf..892af940b3 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/ShardGroup.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/ShardGroup.java @@ -27,7 +27,7 @@ import lombok.Data; /** - * 分片副本组 + * Fragment replica group */ @Data public class ShardGroup { @@ -35,12 +35,12 @@ public class ShardGroup { private List shards = new CopyOnWriteArrayList<>(); private int id; /** - * Leader 任期,leader 切换后递增 = raftNode.leader_term - * 无实际用处 + * Leader term, leader switch increment = raftNode.leader_term + * No practical use */ private long version; /** - * shards 版本号,每次改变后递增 + * shards version number, incremented after each change */ private long confVersion; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Store.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Store.java index 761f35e2e1..25e22ca515 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Store.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/Store.java @@ -34,11 +34,11 @@ public class Store { private String pdAddress; private String raftAddress; private String deployPath; - private String dataPath; // 数据存储路径 + private String dataPath; // Data storage path private int dataVersion; private int partitionCount; private int startTime; - private int usedSize; //rocksdb存储大小 + private int usedSize; // rocksdb storage size private int pdHeartbeatInterval; private Metapb.StoreState state; private Map labels; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java index 0ae018f4cc..cb7a51f003 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java @@ -54,7 +54,7 @@ public List getRaftLocations() { } public Store load() { - // 针对多目录存储的情况下,预先创建文件夹,方便pd端统计文件存储 + // For the case of multi-directory storage, pre-create folders to facilitate pd-side file storage statistics. dataLocations.forEach(path -> { String strPath = Paths.get(path).toAbsolutePath().toString(); File dbFile = new File(strPath); @@ -186,7 +186,7 @@ private String getMinRaftLocation() { } /** - * 获取分区数据存储的位置,如果分布数据不存在,自动创建新的位置 + * Get the location of the partitioned data storage, if distributed data does not exist, automatically create a new location. * * @param partitionId * @return @@ -197,12 +197,12 @@ public String getPartitionStoreLocation(int partitionId, String dbName) { synchronized (this) { location = getPartitionStore(partitionId); if (location == null) { - // 查找分区数最少的存储 + // Find the storage with the least number of partitions location = Metapb.PartitionStore.newBuilder() .setPartitionId(partitionId) .setStoreLocation(getMinDataLocation()) .build(); - // TODO 选择分区数最小的路径 + // TODO Select the path with the least number of partitions. savePartitionStore(location); } } @@ -216,12 +216,12 @@ public String getPartitionRaftLocation(int partitionId) { synchronized (this) { location = getPartitionRaft(partitionId); if (location == null) { - // 查找分区数最少的存储 + // Find the storage with the least number of partitions location = Metapb.PartitionRaft.newBuilder() .setPartitionId(partitionId) .setRaftLocation(getMinRaftLocation()) .build(); - // TODO 选择分区数最小的路径 + // TODO Select the path with the fewest partitions. savePartitionRaft(location); } } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/TaskManager.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/TaskManager.java index ca862e017d..1b9dd659b2 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/TaskManager.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/TaskManager.java @@ -86,9 +86,9 @@ public boolean taskExists(MetaTask.Task task) { } /* - * 判断相同分区下相同任务是否重复 - * partId 分区id - * TaskTypeName 任务类型名称 + * Determine if the same task under the same partition is repeated + * partId partition id + * TaskTypeName Task Type Name * graphName */ public boolean partitionTaskRepeat(int partId, String graphName, String taskTypeName) { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AbstractAsyncTask.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AbstractAsyncTask.java index 640e904b37..87605c0ad1 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AbstractAsyncTask.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AbstractAsyncTask.java @@ -36,7 +36,7 @@ public abstract class AbstractAsyncTask implements AsyncTask, Serializable { private final String graphName; private final String type; /** - * 任务额外需要的参数 + * Parameters needed for the task additionally */ private final Object extra; private AsyncTaskState state; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AsyncTask.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AsyncTask.java index 824d7dddbc..3e2e5fc59a 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AsyncTask.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/AsyncTask.java @@ -20,34 +20,34 @@ public interface AsyncTask { /** - * 需要检查异步任务时候,检查当前的状态,根据状态去做对应的处理 + * Need to check the asynchronous task, check the current status, and handle it accordingly based on the status. */ void handleTask(); /** - * 任务ID + * Task ID */ String getId(); /** - * 针对哪个图的 + * Which graph is it aimed at? */ String getGraphName(); /** - * 针对哪个分区的 + * For which partition */ int getPartitionId(); /** - * 用来进行序列化 + * used for serialization * * @return */ byte[] toBytes(); /** - * 设置执行状态 + * Set execution status * * @param newState */ diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/CleanTask.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/CleanTask.java index acadeebfdb..1d25c0fa81 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/CleanTask.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/asynctask/CleanTask.java @@ -53,7 +53,7 @@ private void cleanTask() { CleanDataRequest request = (CleanDataRequest) getExtra(); var partition = storeEngine.getPartitionManager() .getPartition(getGraphName(), getPartitionId()); - // 只允许清理本分区之外的数据。 缩容等任务会造成干扰, 而且不能删除分区 + // Only allow cleaning data outside of this partition. Tasks such as shrinking can cause interference, and the partition cannot be deleted. if (request.getKeyEnd() == partition.getStartKey() && request.getKeyEnd() == partition.getEndKey() && request.getCleanType() == CleanType.CLEAN_TYPE_EXCLUDE_RANGE && diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/MetaStoreBase.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/MetaStoreBase.java index f97b6d4bca..0d663aa916 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/MetaStoreBase.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/MetaStoreBase.java @@ -32,7 +32,7 @@ import com.google.protobuf.InvalidProtocolBufferException; /** - * Store、Partition等元数据存储到hgstore-metadata图下 + * Store, Partition, etc. metadata are stored under the hgstore-metadata graph. */ public abstract class MetaStoreBase implements Closeable { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/PartitionMetaStore.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/PartitionMetaStore.java index 6a73e04718..948b5ccc27 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/PartitionMetaStore.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/base/PartitionMetaStore.java @@ -20,7 +20,7 @@ import org.apache.hugegraph.rocksdb.access.RocksDBSession; /** - * 元数据存储在分区的default cf中 + * Metadata is stored in the default cf of the partition. */ public class PartitionMetaStore extends MetaStoreBase { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/HgStoreEngineOptions.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/HgStoreEngineOptions.java index 52888c1d18..3b3ff9bc78 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/HgStoreEngineOptions.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/HgStoreEngineOptions.java @@ -36,36 +36,36 @@ public class HgStoreEngineOptions { public static String Raft_Path_Prefix = "raft"; public static String DB_Path_Prefix = "db"; public static String Snapshot_Path_Prefix = "snapshot"; - // store心跳间隔,单位秒 + // store heartbeat interval, unit: seconds private final int storeHBInterval = 30; - // 分区心跳间隔,单位秒 + // Partition heartbeat interval, unit: seconds private final int partitionHBInterval = 5; - // 等待leader超时时间,单位秒 + // Waiting for leader timeout, in seconds private final int waitLeaderTimeout = 30; private final int raftRpcThreadPoolSize = Utils.cpus() * 6; - // 没有PD模式,用于开发调试使用 + // No PD mode, for development and debugging use only private boolean fakePD = false; - // fakePd配置项 + // fakePd configuration items private FakePdOptions fakePdOptions = new FakePdOptions(); private RaftOptions raftOptions = new RaftOptions(); - // pd 服务器地址 + // pd server address private String pdAddress; - // 对外服务地址 + // External service address private String grpcAddress; - // Raft 对外服务地址 + // Raft external service address private String raftAddress; - // 存储路径,支持多个位置,逗号分割 + // Storage path, support multiple locations, separated by commas private String dataPath; private String raftPath; private Map rocksdbConfig; - // 自定义的标签,传给pd + // Custom tags, pass to pd private Map labels; - // Raft任务处理器 + // Raft task processor private RaftTaskHandler taskHandler; private PdProvider pdProvider; - // 数据迁移服务 + // Data Migration Service private DataMover dataTransfer; @Data @@ -93,14 +93,14 @@ public static class RaftOptions { * Install snapshot RPC request default timeout in milliseconds */ private final int rpcInstallSnapshotTimeout = 60 * 60 * 1000; - // 等待leader超时时间,单位秒 + // Wait for leader timeout, in seconds private final int waitLeaderTimeout = 30; /** * The maximum number of entries in AppendEntriesRequest */ private final int maxEntriesSize = 256; /** - * Raft集群发生数据积压后,限速等待时间 单位毫秒 + * Raft cluster data backlog occurs, rate limiting wait time in milliseconds. **/ private final int overloadRateLimit = 100; private final int keepInMemorySegmentCount = 2; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/PartitionEngineOptions.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/PartitionEngineOptions.java index e511b9ba80..001a39631b 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/PartitionEngineOptions.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/PartitionEngineOptions.java @@ -31,18 +31,18 @@ @Data public class PartitionEngineOptions { - // 异步任务执行时间间隔, 单位秒 + // Asynchronous task execution interval, in seconds private final int taskScheduleTime = 60; - // 分裂过程,等待数据对齐超时时间 + // Splitting process, waiting for data alignment timeout private final long splitPartitionTimeout = 30 * 60 * 1000; HgStoreEngineOptions.RaftOptions raftOptions; - // raft存储路径 + // raft storage path private String raftDataPath; private String raftSnapShotPath; private Integer groupId; private String raftAddress; private List peerList; private Configuration conf; - // raft 任务处理器 + // raft task processor private RaftTaskHandler taskHandler; } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/RaftRocksdbOptions.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/RaftRocksdbOptions.java index 56e2f630e9..cb88814936 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/RaftRocksdbOptions.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/options/RaftRocksdbOptions.java @@ -74,7 +74,7 @@ private static void registerRaftRocksdbConfig(HugeConfig options) { DBOptions dbOptions = StorageOptionsFactory.getDefaultRocksDBOptions(); dbOptions.setEnv(rocksdbConfig.getEnv()); - // raft rocksdb数量固定,通过max_write_buffer_number可以控制 + // raft rocksdb number is fixed, can be controlled by max_write_buffer_number //dbOptions.setWriteBufferManager(rocksdbConfig.getBufferManager()); dbOptions.setUnorderedWrite(true); StorageOptionsFactory.registerRocksDBOptions(RocksDBLogStorage.class, diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java index 37de0bae62..6c5840b277 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java @@ -63,7 +63,7 @@ public class DefaultPdProvider implements PdProvider { private PDPulse.Notifier pdPulse; private GraphManager graphManager = null; PDClient.PDEventListener listener = new PDClient.PDEventListener() { - // 监听pd变更信息的listener + // Listening to pd change information listener @Override public void onStoreChanged(NodeEvent event) { if (event.getEventType() == NodeEvent.EventType.NODE_RAFT_CHANGE) { @@ -245,7 +245,7 @@ public void invalidPartitionCache(String graph, int partId) { } /** - * 启动partition心跳流式传输 + * Start partition heartbeat streaming传输 * * @return */ @@ -258,12 +258,12 @@ public boolean startHeartbeatStream(Consumer onError) { public void onNotice(PulseServerNotice response) { PulseResponse content = response.getContent(); - // 消息消费应答,能够正确消费消息,调用accept返回状态码,否则不要调用accept + // Message consumption acknowledgment, if the message can be consumed correctly, call accept to return the status code, otherwise do not call accept. Consumer consumer = integer -> { LOG.debug("Partition heartbeat accept instruction: {}", content); // LOG.info("accept notice id : {}, ts:{}", response.getNoticeId(), System // .currentTimeMillis()); - // http2 并发问题,需要加锁 + // http2 concurrency issue, need to lock // synchronized (pdPulse) { response.ack(); // } @@ -272,7 +272,7 @@ public void onNotice(PulseServerNotice response) { if (content.hasInstructionResponse()) { var pdInstruction = content.getInstructionResponse(); consumer.accept(0); - // 当前的链接变成了follower,重新链接 + // Current link becomes follower, reconnect if (pdInstruction.getInstructionType() == PdInstructionType.CHANGE_TO_FOLLOWER) { onCompleted(); @@ -344,7 +344,7 @@ public void onCompleted() { } /** - * 添加服务端消息监听 + * Add server-side message listening * * @param listener * @return diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/FakePdServiceProvider.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/FakePdServiceProvider.java index 81b48dc199..8c062b8e22 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/FakePdServiceProvider.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/FakePdServiceProvider.java @@ -38,7 +38,7 @@ import lombok.extern.slf4j.Slf4j; /** - * 内置PD服务,用于单机部署或开发调试 + * Built-in PD service, for standalone deployment or development debugging. */ @Slf4j public class FakePdServiceProvider implements PdProvider { @@ -99,7 +99,7 @@ public long registerStore(Store store) throws PDException { log.info("registerStore storeId:{}, storeAddress:{}", store.getId(), store.getStoreAddress()); - // id 不匹配,禁止登录 + // id does not match, login prohibited if (store.getId() != 0 && store.getId() != makeStoreId(store.getStoreAddress())) { throw new PDException(Pdpb.ErrorType.STORE_ID_NOT_EXIST_VALUE, "Store id does not matched"); @@ -132,7 +132,7 @@ public Partition getPartitionByID(String graph, int partId) { Metapb.ShardRole.Follower) // .build(); shards.add(shard); - storeIdx = (storeIdx + 1) >= storeList.size() ? 0 : ++storeIdx; // 顺序选择 + storeIdx = (storeIdx + 1) >= storeList.size() ? 0 : ++storeIdx; // Sequential selection } int partLength = getPartitionLength(); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/PdProvider.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/PdProvider.java index dfdbcaa00a..794c7e4187 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/PdProvider.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/PdProvider.java @@ -76,7 +76,7 @@ public interface PdProvider { void setGraphManager(GraphManager graphManager); /** - * 删除分区 shard group + * Delete partition shard group * * @param groupId */ diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/HgStoreStateMachine.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/HgStoreStateMachine.java index 62666189ce..0f80017c53 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/HgStoreStateMachine.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/HgStoreStateMachine.java @@ -41,7 +41,7 @@ import com.alipay.sofa.jraft.util.Utils; /** - * Raft 状态机 + * Raft State Machine */ public class HgStoreStateMachine extends StateMachineAdapter { @@ -81,7 +81,7 @@ public void onApply(Iterator inter) { try { for (RaftTaskHandler taskHandler : taskHandlers) { if (done != null) { - // Leader分支,本地调用 + // Leader branch, local call if (taskHandler.invoke(groupId, done.op.getOp(), done.op.getReq(), done.closure)) { done.run(Status.OK()); @@ -107,11 +107,11 @@ public void onApply(Iterator inter) { stateListeners.forEach(listener -> { listener.onDataCommitted(committedIndex); }); - // 清理数据 + // Clean up data if (done != null) { done.clear(); } - // 遍历下一条 + // Traverse the next item inter.next(); } } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java index 5ac3ed696a..2e88fe381b 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java @@ -39,16 +39,16 @@ public class RaftOperation { public static final byte SYNC_PARTITION = 0x61; public static final byte BLANK_TASK = 0x62; public static final byte DO_SNAPSHOT = 0x63; - // 集群内部数据迁移操作 + // Cluster internal data migration operation public static final byte IN_WRITE_OP = 0x64; public static final byte IN_CLEAN_OP = 0x65; public static final byte RAFT_UPDATE_PARTITION = 0x66; public static final byte DB_COMPACTION = 0x67; final static byte[] EMPTY_Bytes = new byte[0]; private static final Logger LOG = LoggerFactory.getLogger(RaftOperation.class); - private byte[] values; // req序列化的结果,用于传输给其他raft node - private Object req; // 原始对象,用于本机处理,减少一次反序列化操作 - private byte op; // 操作类型 + private byte[] values; // req serialized result, used for transmitting to other raft nodes + private Object req; // Original object, used for native processing, reducing one deserialization operation + private byte op; // operation type public static RaftOperation create(final byte op) { try { @@ -87,7 +87,7 @@ public static RaftOperation create(final byte op, final Object req) { public static RaftOperation create(final byte op, final com.google.protobuf.GeneratedMessageV3 req) throws IOException { - // 序列化, + // Serialization, final byte[] buffer = new byte[req.getSerializedSize() + 1]; final CodedOutputStream output = CodedOutputStream.newInstance(buffer); output.write(op); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftTaskHandler.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftTaskHandler.java index 5b3b58514c..b641d5ef75 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftTaskHandler.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftTaskHandler.java @@ -20,7 +20,7 @@ import org.apache.hugegraph.store.util.HgStoreException; /** - * 接收raft发送的数据 + * Receive data sent by raft */ public interface RaftTaskHandler { diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/snapshot/HgSnapshotHandler.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/snapshot/HgSnapshotHandler.java index 558e5430d7..eb80b64b4f 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/snapshot/HgSnapshotHandler.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/snapshot/HgSnapshotHandler.java @@ -165,13 +165,13 @@ public void onSnapshotLoad(final SnapshotReader reader, long committedIndex) thr HgStoreException { final String snapshotDir = reader.getPath(); - // 本地保存的快照没必要加载 + // Locally saved snapshots do not need to be loaded if (shouldNotLoad(reader)) { log.info("skip to load snapshot because of should_not_load flag"); return; } - // 直接使用 snapshot + // Directly use snapshot final String graphSnapshotDir = snapshotDir + File.separator + SNAPSHOT_DATA_PATH; log.info("Raft {} begin loadSnapshot, {}", partitionEngine.getGroupId(), graphSnapshotDir); businessHandler.loadSnapshot(graphSnapshotDir, "", partitionEngine.getGroupId(), diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java index cd3ba8705d..ea1cd6cec8 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java @@ -21,7 +21,7 @@ public class HgStoreException extends RuntimeException { public final static int EC_NOEXCEPT = 0; public final static int EC_FAIL = 1000; - //存储的数据格式不支持 + //The data format stored is not supported. public final static int EC_DATAFMT_NOT_SUPPORTED = 1001; public final static int EC_RKDB_CREATE_FAIL = 1201; public final static int EC_RKDB_DOPUT_FAIL = 1202; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/IpUtil.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/IpUtil.java index 8a78cd36e9..ce0ef1a180 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/IpUtil.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/IpUtil.java @@ -34,7 +34,7 @@ public class IpUtil { /** - * 获取所有的ipv4 地址 + * Get all IPv4 addresses * * @return all ipv4 addr * @throws SocketException io error or no network interface @@ -56,7 +56,7 @@ private static List getIpAddress() throws SocketException { } /** - * 根据 option中的raft addr,根据本机的ip获取最相近的一个 + * According to the raft addr in the option, get the closest one based on the local IP. * * @param raftAddress raft addr * @return raft addr that have the nearest distance with given param diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/Version.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/Version.java index 47cefd3dd0..a17bd3722a 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/Version.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/Version.java @@ -27,7 +27,7 @@ public class Version { private static String version = ""; /** - * 软件版本号 + * Software version number */ public static String getVersion() { if (version.isEmpty()) { @@ -43,7 +43,7 @@ public static String getVersion() { } /** - * 存储格式版本号 + * Storage Format Version Number */ public static int getDataFmtVersion() { return 1; From be423cf43c0cbcd32f41f6a898689a3423e77f41 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 6 Aug 2024 20:54:12 +0800 Subject: [PATCH 2/6] fixup --- .../store/client/grpc/KvBatchScanner.java | 3 +- .../store/meta/PartitionManager.java | 31 ------------------- 2 files changed, 1 insertion(+), 33 deletions(-) diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java index 9b4dff13e0..62da3e0aa8 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/KvBatchScanner.java @@ -61,8 +61,7 @@ public class KvBatchScanner implements Closeable { static int maxTaskSizePerStore = PropertyUtil.getInt("net.kv.scanner.task.size", 8); private final StreamObserver sender; // command sender private final KvBatchScannerMerger notifier; // Data notification - 名 -private final String graphName; // graph name + private final String graphName; // graph name private final HgScanQuery scanQuery; private final ScanReceiptRequest.Builder responseBuilder = ScanReceiptRequest.newBuilder(); private final KvBatchReceiver receiver; diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java index c71ae27213..ffd1349a91 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/PartitionManager.java @@ -222,37 +222,6 @@ private void loadPartitions() { // Read partition from data path // Record which partitions - -```python -// 显示所有分区信息 -diskutil list - -// 创建一个新分区 -diskutil partitionDisk disk0 1 GPT HFS+ NewPartition 100% - -// 格式化分区 -diskutil eraseVolume HFS+ NewPartition disk0s3 - -// 列出分区信息 -diskutil info disk0s3 -``` - -```python -// Record which partitions - -```python -// Display all partition information -diskutil list - -// Create a new partition -diskutil partitionDisk disk0 1 GPT HFS+ NewPartition 100% - -// Format the partition -diskutil eraseVolume HFS+ NewPartition disk0s3 - -// List partition information -diskutil info disk0s3 -``` var partIds = new HashSet(); for (String path : this.options.getDataPath().split(",")) { File[] dirs = new File(path + "/" + HgStoreEngineOptions.DB_Path_Prefix).listFiles(); From d1b549136eec443e7c8dabca71d776af1abbe2a1 Mon Sep 17 00:00:00 2001 From: V_Galaxy Date: Tue, 6 Aug 2024 21:08:19 +0800 Subject: [PATCH 3/6] Update hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java --- .../java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java index 1b9e0810ca..eab9c195fa 100644 --- a/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java +++ b/hugegraph-store/hg-store-cli/src/main/java/org/apache/hugegraph/store/cli/loader/HgThread2DB.java @@ -57,7 +57,7 @@ @Slf4j public class HgThread2DB { - /*Total number of tasks in progress and in queue*/ + /* Total number of tasks in progress and in queue */ private static final AtomicInteger taskTotal = new AtomicInteger(0); private static final AtomicInteger queryTaskTotal = new AtomicInteger(0); private static final AtomicLong insertDataCount = new AtomicLong(); From e647cb46de28a436561227a1ad6d9b2ca8e7dd6b Mon Sep 17 00:00:00 2001 From: V_Galaxy Date: Tue, 6 Aug 2024 21:13:06 +0800 Subject: [PATCH 4/6] Apply suggestions from code review --- .../hugegraph/store/client/NodeTxSessionProxy.java | 2 +- .../apache/hugegraph/store/client/grpc/GrpcUtil.java | 4 ++-- .../org/apache/hugegraph/store/HeartbeatService.java | 2 +- .../java/org/apache/hugegraph/store/HgStoreEngine.java | 4 ++-- .../org/apache/hugegraph/store/PartitionEngine.java | 10 +++++----- .../hugegraph/store/business/BusinessHandler.java | 6 +++--- .../hugegraph/store/business/BusinessHandlerImpl.java | 2 +- .../org/apache/hugegraph/store/meta/StoreMetadata.java | 4 ++-- .../org/apache/hugegraph/store/raft/RaftOperation.java | 2 +- .../apache/hugegraph/store/util/HgStoreException.java | 2 +- 10 files changed, 19 insertions(+), 19 deletions(-) diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java index e6d5de8212..c5a6e5c4a4 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/NodeTxSessionProxy.java @@ -692,7 +692,7 @@ public boolean doAction(String table, HgOwnerKey startKey, Integer code, } private List toNodeTkvList(Builder scanReqBuilder) { - // TODO use builder to get owner + // TODO: use builder to get owner String table = scanReqBuilder.getTable(); HgOwnerKey ownerKey = HgStoreClientConst.ALL_PARTITION_OWNER_KEY; byte[] allOwner = ownerKey.getOwner(); diff --git a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java index 313629f1d0..dd7acf2ad8 100644 --- a/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java +++ b/hugegraph-store/hg-store-client/src/main/java/org/apache/hugegraph/store/client/grpc/GrpcUtil.java @@ -63,7 +63,7 @@ static Key.Builder getOwnerKeyBuilder() { Key.Builder builder = keyBuilder.get(); if (builder == null) { builder = Key.newBuilder(); - // TODO Thread-level variables, find the timing for deletion + // TODO: Thread-level variables, find the timing for deletion keyBuilder.set(builder); } return builder; @@ -86,7 +86,7 @@ static Key toKey(HgOwnerKey ownerKey) { Key.Builder builder = keyBuilder.get(); if (builder == null) { builder = Key.newBuilder(); - // TODO Thread-level variables, find the timing for deletion + // TODO: Thread-level variables, find the timing for deletion keyBuilder.set(builder); } return builder diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java index 6f2ca10985..b8fe84ba91 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HeartbeatService.java @@ -48,7 +48,7 @@ public class HeartbeatService implements Lifecycle, PartitionStateListener { private static final int MAX_HEARTBEAT_RETRY_COUNT = 5; // Heartbeat retry count - private static final int REGISTER_RETRY_INTERVAL = 1; //Registration retry interval, in seconds + private static final int REGISTER_RETRY_INTERVAL = 1; // Registration retry interval, in seconds private final HgStoreEngine storeEngine; private final List stateListeners; private final Object partitionThreadLock = new Object(); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java index 72a8becc7f..b76e7a45c9 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/HgStoreEngine.java @@ -343,7 +343,7 @@ private PartitionEngine createPartitionEngine(int groupId, ShardGroup shardGroup /** * Create raft group, in addition to creating the local raft node, also need to notify other peers to create raft nodes. * 1. Traverse partition.shards - Information by storeId + * 2. Retrieve Store information based on storeId * 3. Establish Raft RPC to other stores, send StartRaft messages. * * @param partition @@ -445,7 +445,7 @@ public void deletePartition(Integer groupId, String graphName) { ptEngine.removePartition(graphName); // Delete data businessHandler.deletePartition(graphName, groupId); - //Notify PD to delete partition data + // Notify PD to delete partition data if (ptEngine.isLeader()) { synchronized (this) { partitionManager.deletePartition(graphName, groupId); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java index dd693a9224..bbe5082844 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java @@ -262,7 +262,7 @@ public LogStorage createLogStorage(final String uri, final RaftOptions raftOptio // Check if the peers returned by pd are consistent with the local ones, if not, reset the peerlist if (this.raftNode != null) { - //TODO Check peer list, if peer changes, perform reset + // TODO: Check peer list, if peer changes, perform reset started = true; } @@ -410,7 +410,7 @@ public Status changePeers(List peers, final Closure done) { partitionManager.getPartitionList(getGroupId()), status -> { if (!status.isOk()) { - // TODO What if it fails? + // TODO: What if it fails? log.error("Raft {} destroy node {}" + " error {}", options.getGroupId(), peer, @@ -563,7 +563,7 @@ public Endpoint waitForLeader(long timeOut) { if (partitionManager.isLocalPartition(this.options.getGroupId())) { log.error("Raft {} leader not found, try to repair!", this.options.getGroupId()); - // TODO Check if raft is local, if so, try to fix the Leader, including checking if the configuration is correct. + // TODO: Check if raft is local, if so, try to fix the Leader, including checking if the configuration is correct. storeEngine.createPartitionGroups( partitionManager.getPartitionList(getGroupId()).get(0)); } @@ -630,7 +630,7 @@ public void onStartFollowing(final PeerId newLeaderId, final long newTerm) { public void onConfigurationCommitted(Configuration conf) { try { - //Update shardlist + // Update shardlist log.info("Raft {} onConfigurationCommitted, conf is {}", getGroupId(), conf.toString()); // According to raft endpoint find storeId List peerIds = new ArrayList<>(); @@ -987,7 +987,7 @@ private synchronized void destroyPartitionIfGraphsNull(String graphName, int par storeEngine.deletePartition(partitionId, graphName); } - // without partition engine的情况 + // No partition engine present if (isLeader && partition == null) { partitionManager.deletePartition(graphName, partitionId); } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java index 06175fb7a2..824d4ada77 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandler.java @@ -114,17 +114,17 @@ void loadSnapshot(String snapshotPath, String graph, int partId, long version) t void ingestSstFile(String graph, int partId, Map> sstFiles) throws HgStoreException; - //Submit partition split, delete old data + // Submit partition split, delete old data // Delete partition data boolean deletePartition(String graph, int partId); - //Clean up partitions, delete unnecessary data + // Clean up partitions, delete unnecessary data boolean cleanPartition(String graph, int partId); boolean cleanPartition(String graph, int partId, long startKey, long endKey, CleanType cleanType); - // All table names for all specified partition diagrams + // All table names for all specified partition graph List getTableNames(String graph, int partId); TxBuilder txBuilder(String graph, int partId); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java index d741cd1f19..6421082cf1 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/business/BusinessHandlerImpl.java @@ -719,7 +719,7 @@ public void createTable(String graph, int partId, String table) { @Override public void deleteTable(String graph, int partId, String table) { dropTable(graph, partId, table); - // todo Check if the table is empty, if empty then truly delete the table + // TODO: Check if the table is empty, if empty then truly delete the table // try (RocksDBSession session = getOrCreateGraphDB(graph, partId)) { // session.deleteTables(table); // } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java index cb7a51f003..662b6521f1 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/meta/StoreMetadata.java @@ -202,7 +202,7 @@ public String getPartitionStoreLocation(int partitionId, String dbName) { .setPartitionId(partitionId) .setStoreLocation(getMinDataLocation()) .build(); - // TODO Select the path with the least number of partitions. + // TODO: Select the path with the least number of partitions. savePartitionStore(location); } } @@ -221,7 +221,7 @@ public String getPartitionRaftLocation(int partitionId) { .setPartitionId(partitionId) .setRaftLocation(getMinRaftLocation()) .build(); - // TODO Select the path with the fewest partitions. + // TODO: Select the path with the fewest partitions. savePartitionRaft(location); } } diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java index 2e88fe381b..9ed26b92cb 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/raft/RaftOperation.java @@ -87,7 +87,7 @@ public static RaftOperation create(final byte op, final Object req) { public static RaftOperation create(final byte op, final com.google.protobuf.GeneratedMessageV3 req) throws IOException { - // Serialization, + // Serialization final byte[] buffer = new byte[req.getSerializedSize() + 1]; final CodedOutputStream output = CodedOutputStream.newInstance(buffer); output.write(op); diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java index ea1cd6cec8..b5cef3b353 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/util/HgStoreException.java @@ -21,7 +21,7 @@ public class HgStoreException extends RuntimeException { public final static int EC_NOEXCEPT = 0; public final static int EC_FAIL = 1000; - //The data format stored is not supported. + // The data format stored is not supported. public final static int EC_DATAFMT_NOT_SUPPORTED = 1001; public final static int EC_RKDB_CREATE_FAIL = 1201; public final static int EC_RKDB_DOPUT_FAIL = 1202; From 8658a616ce04ba128425b275963392624b637686 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 6 Aug 2024 21:15:13 +0800 Subject: [PATCH 5/6] fixup --- .../java/org/apache/hugegraph/store/pd/DefaultPdProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java index 6c5840b277..164b43a6c9 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/pd/DefaultPdProvider.java @@ -245,7 +245,7 @@ public void invalidPartitionCache(String graph, int partId) { } /** - * Start partition heartbeat streaming传输 + * Start partition heartbeat streaming transmission * * @return */ From d442f7b6e34b5e17f0829bd102b85e49e3cb695e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 6 Aug 2024 22:55:11 +0800 Subject: [PATCH 6/6] fixup Chinese punctuation --- .../main/java/org/apache/hugegraph/store/PartitionEngine.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java index bbe5082844..11538478cc 100644 --- a/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java +++ b/hugegraph-store/hg-store-core/src/main/java/org/apache/hugegraph/store/PartitionEngine.java @@ -759,7 +759,7 @@ public void doChangeShard(final MetaTask.Task task, Closure done) { HashSet hashSet = new HashSet<>(peers); // Task has the same peers, indicating there is an error in the task itself, task ignored if (peers.size() != hashSet.size()) { - log.info("Raft {} doChangeShard peer is repeat, peers:{}", getGroupId(), + log.info("Raft {} doChangeShard peer is repeat, peers: {}", getGroupId(), peers); } Status result;