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

JiangHua Zhu commented on RATIS-2184:
-------------------------------------

If these exceptions are not caught, there are some exceptions when closing 
MiniRaftCluster.
[org.apache.ratis.grpc.TestRaftWithGrpc-output_03.txt|https://issues.apache.org/jira/secure/attachment/13072924/org.apache.ratis.grpc.TestRaftWithGrpc-output_03.txt]

{code:java}
2024-11-19 17:19:59,361 [main] INFO  impl.MiniRaftCluster 
(MiniRaftCluster.java:shutdown(849)) - ***     Stopping MiniRaftClusterWithGrpc
2024-11-19 17:19:59,362 [main] INFO  impl.MiniRaftCluster 
(MiniRaftCluster.java:shutdown(850)) - *** 
2024-11-19 17:19:59,362 [main] INFO  impl.MiniRaftCluster 
(MiniRaftCluster.java:shutdown(851)) - 
************************************************************** 
2024-11-19 17:19:59,362 [main] INFO  impl.MiniRaftCluster 
(MiniRaftCluster.java:shutdown(852)) - printing ALL groups
  s3:  RUNNING  FOLLOWER s3@group-1F8F98A8697C:t44, leader=s4, voted=s4, 
raftlog=Memoized:s3@group-1F8F98A8697C-SegmentedRaftLog:OPENED:c4854:last(t:44, 
i:4854), conf=conf: {index: 4853, cur=peers:[s3|localhost:15012, 
s4|localhost:15016, s0|localhost:15000, s1|localhost:15004, 
s2|localhost:15008]|listeners:[], old=null} RUNNING
  s4:  RUNNING    LEADER s4@group-1F8F98A8697C:t44, leader=s4, voted=s4, 
raftlog=Memoized:s4@group-1F8F98A8697C-SegmentedRaftLog:OPENED:c4854:last(t:44, 
i:4854), conf=conf: {index: 4853, cur=peers:[s3|localhost:15012, 
s4|localhost:15016, s0|localhost:15000, s1|localhost:15004, 
s2|localhost:15008]|listeners:[], old=null} RUNNING
  s0:  RUNNING  FOLLOWER s0@group-1F8F98A8697C:t44, leader=s4, voted=s4, 
raftlog=Memoized:s0@group-1F8F98A8697C-SegmentedRaftLog:OPENED:c4854:last(t:44, 
i:4854), conf=conf: {index: 4853, cur=peers:[s3|localhost:15012, 
s4|localhost:15016, s0|localhost:15000, s1|localhost:15004, 
s2|localhost:15008]|listeners:[], old=null} RUNNING
  s1:  RUNNING  FOLLOWER s1@group-1F8F98A8697C:t44, leader=s4, voted=s4, 
raftlog=Memoized:s1@group-1F8F98A8697C-SegmentedRaftLog:OPENED:c4854:last(t:44, 
i:4854), conf=conf: {index: 4853, cur=peers:[s3|localhost:15012, 
s4|localhost:15016, s0|localhost:15000, s1|localhost:15004, 
s2|localhost:15008]|listeners:[], old=null} RUNNING
  s2:  RUNNING  FOLLOWER s2@group-1F8F98A8697C:t44, leader=s4, voted=s4, 
raftlog=Memoized:s2@group-1F8F98A8697C-SegmentedRaftLog:OPENED:c4854:last(t:44, 
i:4854), conf=conf: {index: 4853, cur=peers:[s3|localhost:15012, 
s4|localhost:15016, s0|localhost:15000, s1|localhost:15004, 
s2|localhost:15008]|listeners:[], old=null} RUNNING
2024-11-19 17:19:59,364 [main] WARN  impl.MiniRaftCluster 
(MiniRaftCluster.java:shutdown(879)) - shutdown interrupted
java.lang.InterruptedException
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2067)
        at 
java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1475)
        at 
org.apache.ratis.server.impl.MiniRaftCluster.shutdown(MiniRaftCluster.java:868)
        at 
org.apache.ratis.grpc.MiniRaftClusterWithGrpc.shutdown(MiniRaftClusterWithGrpc.java:97)
        at 
org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:152)
        at 
org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:121)
        at org.apache.ratis.RaftBasicTests.testWithLoad(RaftBasicTests.java:338)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
{code}

{code:java}
2024-11-19 17:19:59,365 [main] INFO  impl.MiniRaftCluster 
(MiniRaftCluster.java:shutdown(903)) - MiniRaftClusterWithGrpc shutdown 
completed
2024-11-19 17:19:59,365 [main] INFO  ratis.RaftTestUtil 
(RaftTestUtil.java:gc(596)) - gc 0
2024-11-19 17:20:00,636 [main] WARN  impl.MiniRaftCluster 
(MiniRaftCluster.java:shutdown(909)) - gc interrupted.
java.lang.InterruptedException: sleep interrupted
        at java.lang.Thread.sleep(Native Method)
        at org.apache.ratis.RaftTestUtil.gc(RaftTestUtil.java:598)
        at 
org.apache.ratis.server.impl.MiniRaftCluster.shutdown(MiniRaftCluster.java:907)
        at 
org.apache.ratis.grpc.MiniRaftClusterWithGrpc.shutdown(MiniRaftClusterWithGrpc.java:97)
        at 
org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:152)
        at 
org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:121)
        at org.apache.ratis.RaftBasicTests.testWithLoad(RaftBasicTests.java:338)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.junit.platform.commons.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:766)
        at 
org.junit.jupiter.engine.execution.MethodInvocation.proceed(MethodInvocation.java:60)
{code}

{code:java}
2024-11-19 17:20:06,747 [grpc-default-executor-17] WARN  
server.GrpcServerProtocolService (LogUtils.java:warn(121)) - s3: APPEND_ENTRIES 
onError, lastRequest: s4->s3#3-t20,previous=(t:20
, i:4805),leaderCommit=4805,initializing? false,entries: size=1, first=(t:20, 
i:4806), METADATAENTRY(c:4805): 
org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException: CANCELLED: clien
t cancelled
2024-11-19 17:20:06,726 [grpc-default-executor-68] INFO  server.GrpcLogAppender 
(GrpcLogAppender.java:onError(562)) - s0@group-1F8F98A8697C->s1-GrpcLogAppender 
is already stopped
2024-11-19 17:20:06,747 [grpc-default-executor-35] INFO  server.GrpcLogAppender 
(GrpcLogAppender.java:onError(562)) - s3@group-1F8F98A8697C->s4-GrpcLogAppender 
is already stopped
{code}

I think this is because RaftServerProxy does not have time to release too many 
resources. We can see that after RaftServerImpl is closed, some resources are 
still open.
Therefore, I think it would be better to wait until RaftServerProxy is 
completely closed before ReferenceCountedLeakDetector.getLeakDetector().

> Improve TestRaftWithGrpc test stability
> ---------------------------------------
>
>                 Key: RATIS-2184
>                 URL: https://issues.apache.org/jira/browse/RATIS-2184
>             Project: Ratis
>          Issue Type: Sub-task
>    Affects Versions: 3.0.1
>            Reporter: JiangHua Zhu
>            Assignee: JiangHua Zhu
>            Priority: Major
>         Attachments: image-2024-11-07-15-05-25-681.png, 
> org.apache.ratis.grpc.TestRaftWithGrpc-output_01.txt, 
> org.apache.ratis.grpc.TestRaftWithGrpc-output_02.txt, 
> org.apache.ratis.grpc.TestRaftWithGrpc-output_03.txt, screenshot-1.png
>
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> When running TestRaftWithGrpc, it seems to have poor stability.
> https://github.com/apache/ratis/actions/runs/11591023635/job/32294387846
> The log of the failed run:
> {code:java}
> Error:  org.apache.ratis.grpc.TestRaftWithGrpc.testWithLoad(Boolean)[2]  Time 
> elapsed: 87.315 s  <<< ERROR!
> java.lang.IllegalStateException: #leaks = 4 > 0, #leaks == set.size = 4
>       at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:77)
>       at 
> org.apache.ratis.util.LeakDetector$LeakTrackerSet.assertNoLeaks(LeakDetector.java:100)
>       at 
> org.apache.ratis.util.LeakDetector$LeakTrackerSet.getNumLeaks(LeakDetector.java:94)
>       at 
> org.apache.ratis.util.LeakDetector.assertNoLeaks(LeakDetector.java:174)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster.shutdown(MiniRaftCluster.java:894)
>       at 
> org.apache.ratis.grpc.MiniRaftClusterWithGrpc.shutdown(MiniRaftClusterWithGrpc.java:97)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:152)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:121)
>       at org.apache.ratis.RaftBasicTests.testWithLoad(RaftBasicTests.java:338)
>       at 
> org.apache.ratis.grpc.TestRaftWithGrpc.testWithLoad(TestRaftWithGrpc.java:61)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:175)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
>       at 
> java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:647)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at java.util.ArrayList.forEach(ArrayList.java:1259)
>       at java.util.ArrayList.forEach(ArrayList.java:1259)
> {code}
> {code:java}
> Error:  org.apache.ratis.grpc.TestRaftWithGrpc.testRequestTimeout(Boolean)[1] 
>  Time elapsed: 38.266 s  <<< ERROR!
> java.lang.IllegalStateException: #leaks = 4 > 0, #leaks == set.size = 4
>       at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:77)
>       at 
> org.apache.ratis.util.LeakDetector$LeakTrackerSet.assertNoLeaks(LeakDetector.java:100)
>       at 
> org.apache.ratis.util.LeakDetector$LeakTrackerSet.getNumLeaks(LeakDetector.java:94)
>       at 
> org.apache.ratis.util.LeakDetector.assertNoLeaks(LeakDetector.java:174)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster.shutdown(MiniRaftCluster.java:894)
>       at 
> org.apache.ratis.grpc.MiniRaftClusterWithGrpc.shutdown(MiniRaftClusterWithGrpc.java:97)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:152)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:121)
>       at 
> org.apache.ratis.grpc.TestRaftWithGrpc.testRequestTimeout(TestRaftWithGrpc.java:69)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:175)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
>       at 
> java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:647)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>       at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>       at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>       at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>       at java.util.ArrayList.forEach(ArrayList.java:1259)
>       at java.util.ArrayList.forEach(ArrayList.java:1259)
> {code}



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

Reply via email to