szetszwo commented on code in PR #1177:
URL: https://github.com/apache/ratis/pull/1177#discussion_r1890645618


##########
ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java:
##########
@@ -33,7 +33,6 @@
 public final class ReferenceCountedLeakDetector {
   private static final Logger LOG = 
LoggerFactory.getLogger(ReferenceCountedLeakDetector.class);
   // Leak detection is turned off by default.
-

Review Comment:
   Let's revert this whitespace change.



##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java:
##########
@@ -282,14 +282,17 @@ private void loadLogSegments(long lastIndexInSnapshot,
 
   @Override
   public LogEntryProto get(long index) throws RaftLogIOException {
-    final ReferenceCountedObject<LogEntryProto> ref = retainLog(index);
-    if (ref == null) {
-      return null;
-    }
+    ReferenceCountedObject<LogEntryProto> ref = null;
     try {
+      ref = retainLog(index);

Review Comment:
   Similar to the previous case, `retainLog(index)` should be called before the 
try-block.



##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java:
##########
@@ -281,12 +281,16 @@ public ReferenceCountedObject<LogEntryProto> 
load(LogRecord key) throws IOExcept
       final AtomicReference<ReferenceCountedObject<LogEntryProto>> toReturn = 
new AtomicReference<>();
       final LogSegmentStartEnd startEnd = 
LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen);
       readSegmentFile(file, startEnd, maxOpSize, getLogCorruptionPolicy(), 
raftLogMetrics, entryRef -> {
-        final LogEntryProto entry = entryRef.retain();
-        final TermIndex ti = TermIndex.valueOf(entry);
-        putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE);
-        if (ti.equals(key.getTermIndex())) {
-          toReturn.set(entryRef);
-        } else {
+        try {
+          final LogEntryProto entry = entryRef.retain();

Review Comment:
   `entryRef.retain()` should be called before the try-block.  Otherwise, if it 
throws an exception, we will call `release()` without successfully retained.



##########
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java:
##########
@@ -302,8 +302,21 @@ private long errorWaitTimeMs() {
 
   @Override
   public CompletableFuture<LifeCycle.State> stopAsync() {
-    grpcServerMetrics.unregister();
-    return super.stopAsync();
+    try (AutoCloseableLock ignored = lock.writeLock(caller, LOG::trace)) {
+      if (appendLogRequestObserver != null) {
+        appendLogRequestObserver.stop();
+        appendLogRequestObserver = null;
+      }
+      grpcServerMetrics.unregister();
+      return super.stopAsync();
+    }
+    /*grpcServerMetrics.unregister();
+    CompletableFuture<LifeCycle.State> future = super.stopAsync();
+    if (appendLogRequestObserver != null) {
+      appendLogRequestObserver.stop();
+      appendLogRequestObserver = null;
+    }
+    return future;*/

Review Comment:
   This is a good change.  Could you remove the commented 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to