[
https://issues.apache.org/jira/browse/RATIS-2104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17852235#comment-17852235
]
Duong edited comment on RATIS-2104 at 6/5/24 12:47 AM:
-------------------------------------------------------
Acually, the problem persists after removing the doubled shutdown as per
[https://github.com/apache/ratis/pull/1108.]
Logs advanced tracing shows something weird.
{code:java}
2024-06-04 11:55:27,583 [main] WARN util.ReferenceCountedLeakDetector
(ReferenceCountedLeakDetector.java:lambda$new$0(174)) - LEAK: A
org.apache.ratis.proto.RaftProtos$AppendEntriesRequestProto is not released
properly.
Creation trace:
org.apache.ratis.util.ReferenceCountedObject.wrap(ReferenceCountedObject.java:170)
org.apache.ratis.grpc.server.GrpcServerProtocolService$ServerRequestStreamObserver.onNext(GrpcServerProtocolService.java:151)
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:835)
Retain traces(2):
org.apache.ratis.server.impl.RaftServerProxy.appendEntriesAsync(RaftServerProxy.java:651)
org.apache.ratis.grpc.server.GrpcServerProtocolService$1.process(GrpcServerProtocolService.java:307)
org.apache.ratis.grpc.server.GrpcServerProtocolService$ServerRequestStreamObserver.onNext(GrpcServerProtocolService.java:159)
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:835)
org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1480)
org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$28(RaftServerProxy.java:655)
org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$10(RaftServerImpl.java:887)
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1771)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:835)
Release traces(0):
{code}
it looks like the "finally" block is completely skipped, for example, in
RaftServerProxy.appendEntriesAsync
{code:java}
public CompletableFuture<AppendEntriesReplyProto> appendEntriesAsync(
ReferenceCountedObject<AppendEntriesRequestProto> requestRef) {
AppendEntriesRequestProto request = requestRef.retain();
try {
final RaftGroupId groupId =
ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId());
return getImplFuture(groupId)
.thenCompose(impl -> impl.executeSubmitServerRequestAsync(() ->
impl.appendEntriesAsync(requestRef)));
} finally {
requestRef.release();
}
} {code}
This happens on the pending appendEntries requests when the cluster is shutting
down. I'm not sure how the finally block guarantee can be broken, maybe it's
because the GRPC executor thread is deamon.
was (Author: JIRAUSER290990):
Acually, the problem persists after removing the doubled shutdown as per
[https://github.com/apache/ratis/pull/1108.]
Logs advanced tracing shows something weird.
{code:java}
2024-06-04 11:55:27,583 [main] WARN util.ReferenceCountedLeakDetector
(ReferenceCountedLeakDetector.java:lambda$new$0(174)) - LEAK: A
org.apache.ratis.proto.RaftProtos$AppendEntriesRequestProto is not released
properly.
Creation trace:
org.apache.ratis.util.ReferenceCountedObject.wrap(ReferenceCountedObject.java:170)
org.apache.ratis.grpc.server.GrpcServerProtocolService$ServerRequestStreamObserver.onNext(GrpcServerProtocolService.java:151)
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:835)Retain traces(2):
org.apache.ratis.server.impl.RaftServerProxy.appendEntriesAsync(RaftServerProxy.java:651)
org.apache.ratis.grpc.server.GrpcServerProtocolService$1.process(GrpcServerProtocolService.java:307)
org.apache.ratis.grpc.server.GrpcServerProtocolService$ServerRequestStreamObserver.onNext(GrpcServerProtocolService.java:159)
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:835)org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1480)
org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$28(RaftServerProxy.java:655)
org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$10(RaftServerImpl.java:887)
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1771)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:835) {code}
it looks like the "finally" block is completely skipped, for example, in
RaftServerProxy.appendEntriesAsync
{code:java}
public CompletableFuture<AppendEntriesReplyProto> appendEntriesAsync(
ReferenceCountedObject<AppendEntriesRequestProto> requestRef) {
AppendEntriesRequestProto request = requestRef.retain();
try {
final RaftGroupId groupId =
ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId());
return getImplFuture(groupId)
.thenCompose(impl -> impl.executeSubmitServerRequestAsync(() ->
impl.appendEntriesAsync(requestRef)));
} finally {
requestRef.release();
}
} {code}
This happens on the pending appendEntries requests when the cluster is shutting
down. I'm not sure how the finally block guarantee can be broken, maybe it's
because the GRPC executor thread is deamon.
> 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)