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 15:16:33 UTC

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

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