[ 
https://issues.apache.org/jira/browse/RATIS-2104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17852521#comment-17852521
 ] 

Duong commented on RATIS-2104:
------------------------------

[~szetszwo]  Even if LogAppender is still running in 
MiniRaftCluster.shutdown(), once GrpcServerProtocol receives an appendEntries 
request, it should at least finish processing that request.

I added some naive logs like the following:
{code:java}
public CompletableFuture<AppendEntriesReplyProto> appendEntriesAsync(
    ReferenceCountedObject<AppendEntriesRequestProto> requestRef) {
  AppendEntriesRequestProto request = requestRef.retain();
  LOG.info("Starting {}", request.hashCode());
  try {
    final RaftGroupId groupId = 
ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId());
    return getImplFuture(groupId)
        .thenCompose(impl -> impl.executeSubmitServerRequestAsync(() -> 
impl.appendEntriesAsync(requestRef)));
  } finally {
    LOG.info("Ending {}", request.hashCode());
    requestRef.release();
  }
} {code}
I can confirm that when the MiniRaftCluster is shutting down (JVM doesn't exist 
yet), the "finally" block is sometimes skipped for the last couple of requests 
and the tests fail due to leak detection assertion. 
{code:java}
Starting 1697315093
.... // no Ending
LEAK: A org.apache.ratis.proto.RaftProtos$AppendEntriesRequestProto-1697315093 
is not released properly.{code}
The guarantee that "finally" blocks always run (unless JVM exit) is the basic 
assumption for the correctness around ReferenceCountedObject. 

 

> TestLeaderInstallSnapshot may fail with java.lang.IllegalStateException: 
> allLeaks.size = 4
> ------------------------------------------------------------------------------------------
>
>                 Key: RATIS-2104
>                 URL: https://issues.apache.org/jira/browse/RATIS-2104
>             Project: Ratis
>          Issue Type: Bug
>            Reporter: Tsz-wo Sze
>            Priority: Major
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> I can reproduce it with ratis-3.1.0-rc0
> {code}
> [INFO] Running org.apache.ratis.grpc.TestLeaderInstallSnapshot
>       [ERROR] Tests run: 2, Failures: 0, Errors: 2, Skipped: 0, Time elapsed: 
> 65.892 s <<< FAILURE! - in org.apache.ratis.grpc.TestLeaderInstallSnapshot
> [ERROR] 
> org.apache.ratis.grpc.TestLeaderInstallSnapshot.testInstallSnapshotLeaderSwitch(Boolean)[1]
>   Time elapsed: 20.055 s  <<< ERROR!
> java.lang.IllegalStateException: allLeaks.size = 4
>       at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:77)
>       at 
> org.apache.ratis.util.LeakDetector.assertNoLeaks(LeakDetector.java:107)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster.shutdown(MiniRaftCluster.java:869)
>       at 
> org.apache.ratis.grpc.MiniRaftClusterWithGrpc.shutdown(MiniRaftClusterWithGrpc.java:93)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:149)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:121)
>       at 
> org.apache.ratis.InstallSnapshotFromLeaderTests.testInstallSnapshotLeaderSwitch(InstallSnapshotFromLeaderTests.java:94)
>       at 
> org.apache.ratis.grpc.TestLeaderInstallSnapshot.testInstallSnapshotLeaderSwitch(TestLeaderInstallSnapshot.java:53)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to