You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/02/25 03:18:04 UTC

[GitHub] [ratis] SincereXIA opened a new pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

SincereXIA opened a new pull request #611:
URL: https://github.com/apache/ratis/pull/611


   ## What changes were proposed in this pull request?
   
   This pull request adds a configurable change that allows raft logEntry to not flush to disk immediately after a write. Instead, it will check whether flush is required in the run loop of RaftLogWorker, and we set a minimum flush interval to reduce the number of disk io.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/RATIS-1534
   
   ## How was this patch tested?
   
   We use the ozone freon ockg tool to detect the effect of the changes, testing the command:
   
   ```
   bin/ozone freon ockg  --volume=volume  --bucket=defaultbucket  --thread=10 --number-of-tests=300  --size=`expr 32 \* 1048576` --prefix=stream
   ```
   
   We tested the speed of writing 32MB and 128MB objects before and after enabling asynchronous flush
   
   ![image](https://user-images.githubusercontent.com/20393870/155646950-a2373a74-7ae8-4fba-845f-2b5dddb5143f.png)
   
   
   
   After using async log flush, the average write speed will increase by about 25%
   


-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] SincereXIA commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
SincereXIA commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r815239309



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
##########
@@ -321,6 +326,10 @@ private void run() {
           }
           task.done();
         }
+        if (shouldFlush()) {
+          raftLogMetrics.onRaftLogFlush();
+          flushWrites();
+        }

Review comment:
       Yes, it's better, I've refactored the code




-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] SincereXIA commented on pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
SincereXIA commented on pull request #611:
URL: https://github.com/apache/ratis/pull/611#issuecomment-1050893287


   Hi, @szetszwo,  Really appreciate your suggestion. my code looks much cleaner, thank you!
   
   In addition to what you suggested, I added a line of code to the run loop to check if flush is required, which ensures that the previous logEntry can be flush in time when there are no subsequent tasks. Do you think this is ok?


-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] SincereXIA commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
SincereXIA commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r814808258



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
##########
@@ -321,6 +329,9 @@ private void run() {
           }
           task.done();
         }
+        if (asyncFlushEnabled) {
+          flushToPersistentStore();
+        }

Review comment:
       Yes, it's true




-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] SincereXIA commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
SincereXIA commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r814454458



##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
##########
@@ -348,6 +348,17 @@ static void setForceSyncNum(RaftProperties properties, int forceSyncNum) {
       setInt(properties::setInt, FORCE_SYNC_NUM_KEY, forceSyncNum);
     }
 
+
+    String ASYNC_FLUSH_ENABLE_KEY = PREFIX + ".async.flush";

Review comment:
       ok, i will change it




-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] SincereXIA commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
SincereXIA commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r814805946



##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
##########
@@ -348,6 +348,27 @@ static void setForceSyncNum(RaftProperties properties, int forceSyncNum) {
       setInt(properties::setInt, FORCE_SYNC_NUM_KEY, forceSyncNum);
     }
 
+
+    String ASYNC_FLUSH_ENABLED_KEY = PREFIX + ".async.flush.enabled";
+    boolean ASYNC_FLUSH_ENABLED_DEFAULT = false;
+    static boolean asyncFlushEnabled(RaftProperties properties) {
+      return getBoolean(properties::getBoolean,
+              ASYNC_FLUSH_ENABLED_KEY, ASYNC_FLUSH_ENABLED_DEFAULT, getDefaultLog());
+    }
+    static void setAsyncFlushEnabled(RaftProperties properties, boolean asyncFlush) {
+      setBoolean(properties::setBoolean, ASYNC_FLUSH_ENABLED_KEY, asyncFlush);
+    }
+
+    String ASYNC_FLUSH_MINIMUM_INTERVAL_KEY = PREFIX + ".async.flush.minimum.interval";

Review comment:
       Yes, I will rename it




-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] SincereXIA commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
SincereXIA commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r814454589



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
##########
@@ -374,6 +385,16 @@ private void flushWrites() throws IOException {
     }
   }
 
+  private void flushToPersistentStore() throws IOException {
+    if (shouldFlush() &&
+            lastFlushTimestamp.elapsedTime().compareTo(TimeDuration.valueOf(3, TimeUnit.SECONDS)) > 0) {

Review comment:
       sure, I will make it.




-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] SincereXIA commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
SincereXIA commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r814806459



##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
##########
@@ -348,6 +348,27 @@ static void setForceSyncNum(RaftProperties properties, int forceSyncNum) {
       setInt(properties::setInt, FORCE_SYNC_NUM_KEY, forceSyncNum);
     }
 
+
+    String ASYNC_FLUSH_ENABLED_KEY = PREFIX + ".async.flush.enabled";

Review comment:
       Right, I removed it.




-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] szetszwo merged pull request #611: RATIS-1534. SegmentedRaftLogWorker should enforce a minimum time interval between flush calls

Posted by GitBox <gi...@apache.org>.
szetszwo merged pull request #611:
URL: https://github.com/apache/ratis/pull/611


   


-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] captainzmc commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r814450926



##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
##########
@@ -348,6 +348,17 @@ static void setForceSyncNum(RaftProperties properties, int forceSyncNum) {
       setInt(properties::setInt, FORCE_SYNC_NUM_KEY, forceSyncNum);
     }
 
+
+    String ASYNC_FLUSH_ENABLE_KEY = PREFIX + ".async.flush";

Review comment:
       We can define it to this:
   `String ASYNC_FLUSH_ENABLE_KEY = PREFIX + ".async.flush.enable";`

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
##########
@@ -374,6 +385,16 @@ private void flushWrites() throws IOException {
     }
   }
 
+  private void flushToPersistentStore() throws IOException {
+    if (shouldFlush() &&
+            lastFlushTimestamp.elapsedTime().compareTo(TimeDuration.valueOf(3, TimeUnit.SECONDS)) > 0) {

Review comment:
       Can we make this flush interval configurable as well?




-- 
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: issues-unsubscribe@ratis.apache.org

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



[GitHub] [ratis] szetszwo commented on a change in pull request #611: RATIS-1534. SegmentedRaftLogWorker support async flush

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #611:
URL: https://github.com/apache/ratis/pull/611#discussion_r814852687



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
##########
@@ -321,6 +326,10 @@ private void run() {
           }
           task.done();
         }
+        if (shouldFlush()) {
+          raftLogMetrics.onRaftLogFlush();
+          flushWrites();
+        }

Review comment:
       Good catch.  We should check flush in the loop.  Let's refactor the code a little be as below:
   ```
   @@ -321,6 +326,7 @@ class SegmentedRaftLogWorker {
              }
              task.done();
            }
   +        flushIfNecessary();
          } catch (InterruptedException e) {
            Thread.currentThread().interrupt();
            if (running) {
   @@ -344,13 +350,17 @@ class SegmentedRaftLogWorker {
      }
    
      private boolean shouldFlush() {
   -    return pendingFlushNum >= forceSyncNum ||
   -        (pendingFlushNum > 0 && queue.isEmpty());
   +    if (out == null) {
   +      return false;
   +    } else if (pendingFlushNum >= forceSyncNum) {
   +      return true;
   +    }
   +    return pendingFlushNum > 0 && queue.isEmpty() && lastFlush.elapsedTime().compareTo(flushIntervalMin) > 0;
      }
    
      @SuppressFBWarnings("NP_NULL_PARAM_DEREF")
   -  private void flushWrites() throws IOException {
   -    if (out != null) {
   +  private void flushIfNecessary() throws IOException {
   +    if (shouldFlush()) {
          LOG.debug("{}: flush {}", name, out);
          final Timer.Context timerContext = logFlushTimer.time();
          try {
   @@ -369,6 +379,7 @@ class SegmentedRaftLogWorker {
            }
          } finally {
            timerContext.stop();
   +        lastFlush = Timestamp.currentTime();
          }
          updateFlushedIndexIncreasingly();
        }
   @@ -513,10 +524,7 @@ class SegmentedRaftLogWorker {
          out.write(entry);
          lastWrittenIndex = entry.getIndex();
          pendingFlushNum++;
   -      if (shouldFlush()) {
   -        raftLogMetrics.onRaftLogFlush();
   -        flushWrites();
   -      }
   +      flushIfNecessary();
        }
   ```
   




-- 
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: issues-unsubscribe@ratis.apache.org

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