You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2022/09/13 16:06:24 UTC

[GitHub] [iotdb] SzyWilliam commented on a diff in pull request #7286: [IOTDB-4110] customize trigger snapshot

SzyWilliam commented on code in PR #7286:
URL: https://github.com/apache/iotdb/pull/7286#discussion_r969808594


##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java:
##########
@@ -575,6 +583,50 @@ public ConsensusGenericResponse triggerSnapshot(ConsensusGroupId groupId) {
     return ConsensusGenericResponse.newBuilder().setSuccess(reply.isSuccess()).build();
   }
 
+  private void triggerSnapshotByCustomize(ConsensusConfig config) {
+    Iterable<RaftGroupId> groupIds = server.getGroupIds();
+
+    while (groupIds.iterator().hasNext()) {
+      RaftGroupId raftGroupId = groupIds.iterator().next();
+      File currentDir = null;
+
+      try {
+         currentDir = server.getDivision(raftGroupId).getRaftStorage().getStorageDir()
+                .getCurrentDir();
+      }
+      catch (IOException e) {
+        failed(new RatisGetDivisionException(e));

Review Comment:
   failed(...) is called when construct a fail reply message to client. Just use log.warn here to warn user of GetDivisionException.



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java:
##########
@@ -575,6 +583,50 @@ public ConsensusGenericResponse triggerSnapshot(ConsensusGroupId groupId) {
     return ConsensusGenericResponse.newBuilder().setSuccess(reply.isSuccess()).build();
   }
 
+  private void triggerSnapshotByCustomize(ConsensusConfig config) {
+    Iterable<RaftGroupId> groupIds = server.getGroupIds();
+
+    while (groupIds.iterator().hasNext()) {
+      RaftGroupId raftGroupId = groupIds.iterator().next();
+      File currentDir = null;
+
+      try {
+         currentDir = server.getDivision(raftGroupId).getRaftStorage().getStorageDir()
+                .getCurrentDir();
+      }
+      catch (IOException e) {
+        failed(new RatisGetDivisionException(e));
+      }
+
+      long currentDirLength = currentDir.length();

Review Comment:
   File.length() will return 
   > Returns the length of the file denoted by this abstract pathname. The return value is unspecified if this pathname denotes a directory.
   
   Not the directory total size. Please refer to java doc https://docs.oracle.com/javase/8/docs/api/java/io/File.html#length--



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java:
##########
@@ -575,6 +583,50 @@ public ConsensusGenericResponse triggerSnapshot(ConsensusGroupId groupId) {
     return ConsensusGenericResponse.newBuilder().setSuccess(reply.isSuccess()).build();
   }
 
+  private void triggerSnapshotByCustomize(ConsensusConfig config) {

Review Comment:
   maybe we should add synchronized to method signature. Consider this: this method is not finished when 60s past and another call started.



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java:
##########
@@ -575,6 +583,50 @@ public ConsensusGenericResponse triggerSnapshot(ConsensusGroupId groupId) {
     return ConsensusGenericResponse.newBuilder().setSuccess(reply.isSuccess()).build();
   }
 
+  private void triggerSnapshotByCustomize(ConsensusConfig config) {
+    Iterable<RaftGroupId> groupIds = server.getGroupIds();
+
+    while (groupIds.iterator().hasNext()) {
+      RaftGroupId raftGroupId = groupIds.iterator().next();
+      File currentDir = null;
+
+      try {
+         currentDir = server.getDivision(raftGroupId).getRaftStorage().getStorageDir()
+                .getCurrentDir();
+      }
+      catch (IOException e) {
+        failed(new RatisGetDivisionException(e));
+      }
+
+      long currentDirLength = currentDir.length();
+      long triggerSnapshotFileSize = config.getRatisConfig().getSnapshot().getTriggerSnapshotFileSize();
+
+      if (currentDirLength >= triggerSnapshotFileSize) {
+        ConsensusGenericResponse consensusGenericResponse = triggerSnapshot(Utils.fromRaftGroupIdToConsensusGroupId(raftGroupId));
+        if (consensusGenericResponse.isSuccess()){
+          logger.info("Raft group " + raftGroupId + " took snapshot successfully");
+        }
+        else {
+          logger.warn("Raft group " + raftGroupId + " failed to take snapshot");
+        }
+      }
+    }
+  }
+
+  private void createSnapshotThread(ConsensusConfig config) {
+    ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+    Runnable command = () -> triggerSnapshotByCustomize(config);
+    long delay = config.getRatisConfig().getSnapshot().getTriggerSnapshotTime();
+
+    ScheduledExecutorUtil.safelyScheduleWithFixedDelay(
+            executor,
+            command,

Review Comment:
   Use this::triggerSnapshotByCustomize instead of lambda.



##########
consensus/src/main/java/org/apache/iotdb/consensus/config/RatisConfig.java:
##########
@@ -295,10 +309,12 @@ public static class Builder {
       private long autoTriggerThreshold =
           RaftServerConfigKeys.Snapshot.AUTO_TRIGGER_THRESHOLD_DEFAULT;
       private int retentionFileNum = RaftServerConfigKeys.Snapshot.RETENTION_FILE_NUM_DEFAULT;
+      private long triggerSnapshotTime = 60;
+      private long triggerSnapshotFileSize = 20L << 30;
 

Review Comment:
   Add comments here. 60 seconds and 20 GB.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org