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

JiangHua Zhu edited comment on RATIS-2184 at 11/12/24 7:49 AM:
---------------------------------------------------------------

[~szetszwo], I have been paying attention to this issue these days.
Currently, 2 problems have been found, both of which will affect the stability 
of TestRaftWithGrpc test.
1. GrpcLogAppender.appendLog()->nextAppendEntriesRequest() triggers an 
exception.
Log:
{code:java}
2024-11-12 15:30:58,758 
[s2@group-4D4EA4C133DD->s4-GrpcLogAppender-LogAppenderDaemon] WARN  
leader.LogAppenderDaemon (LogAppenderDaemon.java:run(89)) - 
s2@group-4D4EA4C133DD->s4-GrpcLogAppender-LogAppenderDaemon failed
org.apache.ratis.server.raftlog.RaftLogIOException: Log entry not found: index 
= 4269
        at 
org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog.retainEntryWithData(SegmentedRaftLog.java:334)
        at 
org.apache.ratis.server.leader.LogAppenderBase.nextAppendEntriesRequest(LogAppenderBase.java:264)
        at 
org.apache.ratis.grpc.server.GrpcLogAppender.appendLog(GrpcLogAppender.java:387)
        at 
org.apache.ratis.grpc.server.GrpcLogAppender.run(GrpcLogAppender.java:262)
        at 
org.apache.ratis.server.leader.LogAppenderDaemon.run(LogAppenderDaemon.java:80)
        at java.lang.Thread.run(Thread.java:750)
{code}
 !screenshot-1.png! 

This will cause entryWithData to not call the release() method, which is a bug. 
We need to catch this exception and execute release().
2. Under the current jvm parameter settings, TestRaftWithGrpc will also have a 
certain probability of failure.
In addition, after testing, it was found that LogSegment#entryCache is normally 
closed.



was (Author: jianghuazhu):
[~szetszwo], I have been paying attention to this issue these days.
Currently, 2 problems have been found, both of which will affect the stability 
of TestRaftWithGrpc test.
1. GrpcLogAppender.appendLog()->nextAppendEntriesRequest() triggers an 
exception.
Log:
{code:java}
2024-11-12 15:30:58,758 
[s2@group-4D4EA4C133DD->s4-GrpcLogAppender-LogAppenderDaemon] WARN  
leader.LogAppenderDaemon (LogAppenderDaemon.java:run(89)) - 
s2@group-4D4EA4C133DD->s4-GrpcLogAppender-LogAppenderDaemon failed
org.apache.ratis.server.raftlog.RaftLogIOException: Log entry not found: index 
= 4269
        at 
org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog.retainEntryWithData(SegmentedRaftLog.java:334)
        at 
org.apache.ratis.server.leader.LogAppenderBase.nextAppendEntriesRequest(LogAppenderBase.java:264)
        at 
org.apache.ratis.grpc.server.GrpcLogAppender.appendLog(GrpcLogAppender.java:387)
        at 
org.apache.ratis.grpc.server.GrpcLogAppender.run(GrpcLogAppender.java:262)
        at 
org.apache.ratis.server.leader.LogAppenderDaemon.run(LogAppenderDaemon.java:80)
        at java.lang.Thread.run(Thread.java:750)
{code}
 !screenshot-1.png! 

This will cause pending to not call the release() method, which is a bug. We 
need to catch this exception and execute release().
2. Under the current jvm parameter settings, TestRaftWithGrpc will also have a 
certain probability of failure.
In addition, after testing, it was found that LogSegment#entryCache is normally 
closed.


> 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, screenshot-1.png
>
>
> 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