This is an automated email from the ASF dual-hosted git repository.
szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git
The following commit(s) were added to refs/heads/master by this push:
new be28b3907 RATIS-1890. SegmentedRaftLogCache#shouldEvict should only
iterate over closed segments once (#921)
be28b3907 is described below
commit be28b3907f4fee8957fb2824770e4925364d0a8f
Author: Ivan Andika <[email protected]>
AuthorDate: Tue Sep 19 23:39:45 2023 +0800
RATIS-1890. SegmentedRaftLogCache#shouldEvict should only iterate over
closed segments once (#921)
---
.../raftlog/segmented/SegmentedRaftLogCache.java | 50 ++++++++++++++++++----
1 file changed, 42 insertions(+), 8 deletions(-)
diff --git
a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java
b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java
index 6d5aa6d41..bd6d83139 100644
---
a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java
+++
b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java
@@ -157,6 +157,38 @@ public class SegmentedRaftLogCache {
}
}
+ private static class CacheInfo {
+ static CacheInfo get(List<LogSegment> list) {
+ long size = 0L;
+ long count = 0L;
+ for (LogSegment segment: list) {
+ if (segment.hasCache()) {
+ count++;
+ size += segment.getTotalCacheSize();
+ }
+ }
+ return new CacheInfo(size, count);
+ }
+
+ /** Total cache size in bytes. */
+ private final long size;
+ /** The number of cached segments. */
+ private final long count;
+
+ CacheInfo(long size, long count) {
+ this.size = size;
+ this.count = count;
+ }
+
+ public long getSize() {
+ return size;
+ }
+
+ public long getCount() {
+ return count;
+ }
+ }
+
static class LogSegmentList {
private final Object name;
private final List<LogSegment> segments = new CopyOnWriteArrayList<>();
@@ -191,8 +223,8 @@ public class SegmentedRaftLogCache {
return sizeInBytes;
}
- long getTotalCacheSize() {
- return segments.stream().mapToLong(LogSegment::getTotalCacheSize).sum();
+ CacheInfo getCacheInfo() {
+ return CacheInfo.get(segments);
}
long countCached() {
@@ -410,13 +442,15 @@ public class SegmentedRaftLogCache {
return openSegment == null ? 0 : openSegment.getTotalFileSize();
}
- private long getTotalCacheSize() {
- return closedSegments.getTotalCacheSize() +
-
Optional.ofNullable(openSegment).map(LogSegment::getTotalCacheSize).orElse(0L);
- }
-
boolean shouldEvict() {
- return closedSegments.countCached() > maxCachedSegments ||
getTotalCacheSize() > maxSegmentCacheSize;
+ final CacheInfo closedSegmentsCacheInfo = closedSegments.getCacheInfo();
+ if (closedSegmentsCacheInfo.getCount() > maxCachedSegments) {
+ return true;
+ }
+
+ final long size = closedSegmentsCacheInfo.getSize()
+ +
Optional.ofNullable(openSegment).map(LogSegment::getTotalCacheSize).orElse(0L);
+ return size > maxSegmentCacheSize;
}
void evictCache(long[] followerIndices, long safeEvictIndex, long
lastAppliedIndex) {