Skip to content

Conversation

@ivandika3
Copy link
Contributor

What changes were proposed in this pull request?

Please refer to https://issues.apache.org/jira/browse/RATIS-2186

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/RATIS-2186

How was this patch tested?

Unit test.

@ivandika3
Copy link
Contributor Author

cc: @szetszwo @SzyWilliam

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ivandika3 , thanks a lot for working on this!

  • Let's also change SegmentedRaftLogCache to handle the -1 case, which is the same as the toDelete list being empty.
+++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java
@@ -364,6 +364,9 @@ public class SegmentedRaftLogCache {
           list.addAll(segments);
           segments.clear();
           sizeInBytes = 0;
+        } else if (segmentIndex == -1) {
+          // nothing to purge
+          return null;
         } else if (segmentIndex >= 0) {
           // we start to purge the closedSegments which do not overlap with index.
           LogSegment overlappedSegment = segments.get(segmentIndex);

Comment on lines 329 to 335
long startIndex = getStartIndex();
if (suggestedIndex < startIndex) {
LOG.info("{}: purge is skipped since the suggested index {} is lower than " +
"log start index {}",
getName(), suggestedIndex, startIndex);
return CompletableFuture.completedFuture(lastPurge);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The change looks good. Could you also

  • add an adjustedIndex (instead of the original code updating the suggestedIndex and having finalSuggestedIndex) and
  • add more info to the message?
+++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java
@@ -318,20 +318,28 @@ public abstract class RaftLogBase implements RaftLog {
 
   @Override
   public final CompletableFuture<Long> purge(long suggestedIndex) {
+    final long adjustedIndex;
     if (purgePreservation > 0) {
       final long currentIndex = getNextIndex() - 1;
-      suggestedIndex = Math.min(suggestedIndex, currentIndex - purgePreservation);
+      adjustedIndex = Math.min(suggestedIndex, currentIndex - purgePreservation);
+    } else {
+      adjustedIndex = suggestedIndex;
     }
     final long lastPurge = purgeIndex.get();
-    if (suggestedIndex - lastPurge < purgeGap) {
+    if (adjustedIndex - lastPurge < purgeGap) {
+      return CompletableFuture.completedFuture(lastPurge);
+    }
+    final long startIndex = getStartIndex();
+    if (adjustedIndex < startIndex) {
+      LOG.info("{}: purge({}) is skipped: adjustedIndex = {} < startIndex = {}, purgePreservation = {}",
+          getName(), suggestedIndex, adjustedIndex, startIndex, purgePreservation);
       return CompletableFuture.completedFuture(lastPurge);
     }
-    LOG.info("{}: purge {}", getName(), suggestedIndex);
-    final long finalSuggestedIndex = suggestedIndex;
-    return purgeImpl(suggestedIndex).whenComplete((purged, e) -> {
+    LOG.info("{}: purge {}", getName(), adjustedIndex);
+    return purgeImpl(adjustedIndex).whenComplete((purged, e) -> {
       updatePurgeIndex(purged);
       if (e != null) {
-        LOG.warn(getName() + ": Failed to purge " + finalSuggestedIndex, e);
+        LOG.warn(getName() + ": Failed to purge " + adjustedIndex, e);
       }
     });
   }

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the review. A lot clearer now. Updated as per the suggestion.

}

private SegmentedRaftLog newSegmentedRaftLogWithSnapshotIndex(RaftStorage storage, RaftProperties properties,
SegmentedRaftLog newSegmentedRaftLogWithSnapshotIndex(RaftStorage storage, RaftProperties properties,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Keep it private.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. Updated.

Comment on lines 621 to 622
private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int purgeGap, long purgeIndex,
long expectedIndex) throws Exception {
List<SegmentRange> ranges = prepareRanges(startTerm, endTerm, segmentSize, 0);
long expectedIndex, long startIndex, long purgePreservation) throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's overload the method. Then, we don't have to change the other calls.

   private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int purgeGap, long purgeIndex,
       long expectedIndex) throws Exception {
-    List<SegmentRange> ranges = prepareRanges(startTerm, endTerm, segmentSize, 0);
+    purgeAndVerify(startTerm, endTerm, segmentSize, purgeGap, purgeIndex, expectedIndex, 0, 0);
+  }
+
+  private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int purgeGap, long purgeIndex,
+      long expectedIndex, long startIndex, long purgePreservation) throws Exception {
+    List<SegmentRange> ranges = prepareRanges(startTerm, endTerm, segmentSize, startIndex);
     List<LogEntryProto> entries = prepareLogEntries(ranges, null);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. Updated.

@ivandika3
Copy link
Contributor Author

ivandika3 commented Nov 12, 2024

@szetszwo Thanks for the review and the suggestions. I have updated them accordingly.

Let's also change SegmentedRaftLogCache to handle the -1 case, which is the same as the toDelete list being empty.

This is a good idea to prevent similar issue when calling SegmentedRaftLogCache#purge

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 the change looks good.

@szetszwo szetszwo merged commit 13b8cdd into apache:master Nov 12, 2024
@szetszwo
Copy link
Contributor

@ivandika3 , thanks a lot for working on this!

@OneSizeFitsQuorum , thanks for reviewing this!

@ivandika3
Copy link
Contributor Author

@szetszwo @OneSizeFitsQuorum Thanks for the reviews and suggestions.

OneSizeFitsQuorum pushed a commit to OneSizeFitsQuorum/ratis that referenced this pull request Nov 18, 2024
SzyWilliam pushed a commit to SzyWilliam/ratis that referenced this pull request Jan 9, 2025
szetszwo pushed a commit to szetszwo/ratis that referenced this pull request May 23, 2025
szetszwo pushed a commit to szetszwo/ratis that referenced this pull request May 23, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants