[ https://issues.apache.org/jira/browse/RATIS-706?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Mukul Kumar Singh updated RATIS-706: ------------------------------------ Description: I started an Ozone cluster on Kubernetes and started a freon test (ozone freon ockg -n10000) After a while I found that the one freon instance is not creating keys any more. I checked the om RPC endpoint with ozone insight and no RPC messages has been arrived. Based on the jstack output we have a deadlock between PeerProxyMap.handleException and GrpcClientRpc.sendRequestAsync. I am not sure (yet) what is the exact problem, but based on the stack traces It seems to be Ratis related. {code} Found one Java-level deadlock: ============================= "pool-2-thread-6": waiting to lock monitor 0x00007f80356c8800 (object 0x000000033eb70a00, a java.lang.Object), which is held by "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty queue]" "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty queue]": waiting to lock monitor 0x0000000001170980 (object 0x000000033eb99b10, a org.apache.ratis.util.SlidingWindow$Client), which is held by "java.util.concurrent.ThreadPoolExecutor$Worker@df368f8[State = -1, empty queue]" "java.util.concurrent.ThreadPoolExecutor$Worker@df368f8[State = -1, empty queue]": waiting to lock monitor 0x00007f80356c8800 (object 0x000000033eb70a00, a java.lang.Object), which is held by "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty queue]" Java stack information for the threads listed above: =================================================== "pool-2-thread-6": at org.apache.ratis.util.PeerProxyMap.getProxy(PeerProxyMap.java:103) - waiting to lock <0x000000033eb70a00> (a java.lang.Object) at org.apache.ratis.grpc.client.GrpcClientRpc.sendRequestAsyncUnordered(GrpcClientRpc.java:78) at org.apache.ratis.client.impl.UnorderedAsync.sendRequestWithRetry(UnorderedAsync.java:75) at org.apache.ratis.client.impl.UnorderedAsync.send(UnorderedAsync.java:59) at org.apache.ratis.client.impl.RaftClientImpl.sendWatchAsync(RaftClientImpl.java:139) at org.apache.hadoop.hdds.scm.XceiverClientRatis.watchForCommit(XceiverClientRatis.java:282) at org.apache.hadoop.hdds.scm.storage.CommitWatcher.watchForCommit(CommitWatcher.java:198) at org.apache.hadoop.hdds.scm.storage.CommitWatcher.watchOnLastIndex(CommitWatcher.java:161) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.watchForCommit(BlockOutputStream.java:346) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:482) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.close(BlockOutputStream.java:496) at org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.close(BlockOutputStreamEntry.java:143) at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:435) at org.apache.hadoop.ozone.client.io.KeyOutputStream.close(KeyOutputStream.java:473) at org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:60) - locked <0x00000003f2ba4240> (a org.apache.hadoop.ozone.client.io.OzoneOutputStream) at org.apache.hadoop.ozone.freon.RandomKeyGenerator.createKey(RandomKeyGenerator.java:710) at org.apache.hadoop.ozone.freon.RandomKeyGenerator.access$1100(RandomKeyGenerator.java:88) at org.apache.hadoop.ozone.freon.RandomKeyGenerator$ObjectCreator.run(RandomKeyGenerator.java:615) at java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.3/Executors.java:515) at java.util.concurrent.FutureTask.run(java.base@11.0.3/FutureTask.java:264) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.3/ThreadPoolExecutor.java:1128) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.3/ThreadPoolExecutor.java:628) at java.lang.Thread.run(java.base@11.0.3/Thread.java:834) "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty queue]": at org.apache.ratis.util.SlidingWindow$Client.resetFirstSeqNum(SlidingWindow.java:348) - waiting to lock <0x000000033eb99b10> (a org.apache.ratis.util.SlidingWindow$Client) at org.apache.ratis.client.impl.OrderedAsync.resetSlidingWindow(OrderedAsync.java:121) at org.apache.ratis.client.impl.OrderedAsync$$Lambda$496/0x0000000840498440.accept(Unknown Source) at org.apache.ratis.client.impl.RaftClientImpl.lambda$handleIOException$6(RaftClientImpl.java:349) at org.apache.ratis.client.impl.RaftClientImpl$$Lambda$427/0x00000008402e1840.accept(Unknown Source) at java.util.Optional.ifPresent(java.base@11.0.3/Optional.java:183) at org.apache.ratis.client.impl.RaftClientImpl.handleIOException(RaftClientImpl.java:349) at org.apache.ratis.client.impl.OrderedAsync.lambda$sendRequest$10(OrderedAsync.java:236) at org.apache.ratis.client.impl.OrderedAsync$$Lambda$358/0x00000008402d7040.apply(Unknown Source) at java.util.concurrent.CompletableFuture.uniExceptionally(java.base@11.0.3/CompletableFuture.java:986) at java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(java.base@11.0.3/CompletableFuture.java:970) at java.util.concurrent.CompletableFuture.postComplete(java.base@11.0.3/CompletableFuture.java:506) at java.util.concurrent.CompletableFuture.completeExceptionally(java.base@11.0.3/CompletableFuture.java:2088) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.completeReplyExceptionally(GrpcClientProtocolClient.java:345) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.close(GrpcClientProtocolClient.java:334) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.access$400(GrpcClientProtocolClient.java:261) at org.apache.ratis.grpc.client.GrpcClientProtocolClient.lambda$close$1(GrpcClientProtocolClient.java:141) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$$Lambda$435/0x00000008402d1c40.accept(Unknown Source) at java.util.Optional.ifPresent(java.base@11.0.3/Optional.java:183) at org.apache.ratis.grpc.client.GrpcClientProtocolClient.close(GrpcClientProtocolClient.java:141) at org.apache.ratis.util.PeerProxyMap$PeerAndProxy.lambda$close$1(PeerProxyMap.java:74) at org.apache.ratis.util.PeerProxyMap$PeerAndProxy$$Lambda$430/0x00000008402e1440.run(Unknown Source) at org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$2(LifeCycle.java:231) at org.apache.ratis.util.LifeCycle$$Lambda$433/0x00000008402d1040.get(Unknown Source) at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:251) at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:229) at org.apache.ratis.util.PeerProxyMap$PeerAndProxy.close(PeerProxyMap.java:70) - locked <0x000000008c1ba0e8> (a org.apache.ratis.util.PeerProxyMap$PeerAndProxy) at org.apache.ratis.util.PeerProxyMap.resetProxy(PeerProxyMap.java:127) - locked <0x000000033eb70a00> (a java.lang.Object) at org.apache.ratis.util.PeerProxyMap.handleException(PeerProxyMap.java:136) at org.apache.ratis.client.impl.RaftClientRpcWithProxy.handleException(RaftClientRpcWithProxy.java:47) at org.apache.ratis.client.impl.RaftClientImpl.handleIOException(RaftClientImpl.java:372) at org.apache.ratis.client.impl.OrderedAsync.lambda$sendRequest$10(OrderedAsync.java:236) at org.apache.ratis.client.impl.OrderedAsync$$Lambda$358/0x00000008402d7040.apply(Unknown Source) at java.util.concurrent.CompletableFuture.uniExceptionally(java.base@11.0.3/CompletableFuture.java:986) at java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(java.base@11.0.3/CompletableFuture.java:970) at java.util.concurrent.CompletableFuture.postComplete(java.base@11.0.3/CompletableFuture.java:506) at java.util.concurrent.CompletableFuture.completeExceptionally(java.base@11.0.3/CompletableFuture.java:2088) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$timeoutCheck$3(GrpcClientProtocolClient.java:324) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers$$Lambda$490/0x000000084049f040.accept(Unknown Source) at java.util.Optional.ifPresent(java.base@11.0.3/Optional.java:183) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.handleReplyFuture(GrpcClientProtocolClient.java:329) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.timeoutCheck(GrpcClientProtocolClient.java:324) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$onNext$1(GrpcClientProtocolClient.java:318) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers$$Lambda$336/0x00000008403b2040.run(Unknown Source) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:113) at org.apache.ratis.util.TimeoutScheduler$$Lambda$345/0x00000008403ac440.accept(Unknown Source) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:133) at org.apache.ratis.util.TimeoutScheduler$$Lambda$349/0x00000008403a1440.run(Unknown Source) at org.apache.ratis.util.LogUtils$1$$Lambda$448/0x00000008402d6840.run(Unknown Source) at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:50) at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:91) at java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.3/Executors.java:515) at java.util.concurrent.FutureTask.run(java.base@11.0.3/FutureTask.java:264) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@11.0.3/ScheduledThreadPoolExecutor.java:304) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.3/ThreadPoolExecutor.java:1128) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.3/ThreadPoolExecutor.java:628) at java.lang.Thread.run(java.base@11.0.3/Thread.java:834) "java.util.concurrent.ThreadPoolExecutor$Worker@df368f8[State = -1, empty queue]": at org.apache.ratis.util.PeerProxyMap.getProxy(PeerProxyMap.java:103) - waiting to lock <0x000000033eb70a00> (a java.lang.Object) at org.apache.ratis.grpc.client.GrpcClientRpc.sendRequestAsync(GrpcClientRpc.java:66) at org.apache.ratis.client.impl.OrderedAsync.sendRequest(OrderedAsync.java:208) at org.apache.ratis.client.impl.OrderedAsync.sendRequestWithRetry(OrderedAsync.java:169) at org.apache.ratis.client.impl.OrderedAsync$$Lambda$449/0x00000008402d6c40.accept(Unknown Source) at org.apache.ratis.util.SlidingWindow$Client.sendOrDelayRequest(SlidingWindow.java:278) at org.apache.ratis.util.SlidingWindow$Client.retry(SlidingWindow.java:294) - locked <0x000000033eb99b10> (a org.apache.ratis.util.SlidingWindow$Client) at org.apache.ratis.client.impl.OrderedAsync.lambda$scheduleWithTimeout$7(OrderedAsync.java:195) at org.apache.ratis.client.impl.OrderedAsync$$Lambda$444/0x00000008402d4840.run(Unknown Source) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:113) at org.apache.ratis.util.TimeoutScheduler$$Lambda$345/0x00000008403ac440.accept(Unknown Source) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:133) at org.apache.ratis.util.TimeoutScheduler$$Lambda$349/0x00000008403a1440.run(Unknown Source) at org.apache.ratis.util.LogUtils$1$$Lambda$448/0x00000008402d6840.run(Unknown Source) at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:50) at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:91) at java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.3/Executors.java:515) at java.util.concurrent.FutureTask.run(java.base@11.0.3/FutureTask.java:264) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@11.0.3/ScheduledThreadPoolExecutor.java:304) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.3/ThreadPoolExecutor.java:1128) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.3/ThreadPoolExecutor.java:628) at java.lang.Thread.run(java.base@11.0.3/Thread.java:834) Found 1 deadlock. {code} was: I started an Ozone cluster on Kubernetes and started a freon test (ozone freon ockg -n10000) After a while I found that the one freon instance is not creating keys any more. I checked the om RPC endpoint with ozone insight and no RPC messages has been arrived. Based on the jstack output we have a deadlock between PeerProxyMap.handleException and GrpcClientRpc.sendRequestAsync. I am not sure (yet) what is the exact problem, but based on the stack traces It seems to be Ratis related. > Dead lock in GrpcClientRpc > -------------------------- > > Key: RATIS-706 > URL: https://issues.apache.org/jira/browse/RATIS-706 > Project: Ratis > Issue Type: Bug > Components: gRPC > Reporter: Marton Elek > Priority: Major > Attachments: jstack.txt > > > I started an Ozone cluster on Kubernetes and started a freon test (ozone > freon ockg -n10000) > After a while I found that the one freon instance is not creating keys any > more. I checked the om RPC endpoint with ozone insight and no RPC messages > has been arrived. > Based on the jstack output we have a deadlock between > PeerProxyMap.handleException and GrpcClientRpc.sendRequestAsync. > I am not sure (yet) what is the exact problem, but based on the stack traces > It seems to be Ratis related. > {code} > Found one Java-level deadlock: > ============================= > "pool-2-thread-6": > waiting to lock monitor 0x00007f80356c8800 (object 0x000000033eb70a00, a > java.lang.Object), > which is held by > "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty > queue]" > "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty > queue]": > waiting to lock monitor 0x0000000001170980 (object 0x000000033eb99b10, a > org.apache.ratis.util.SlidingWindow$Client), > which is held by > "java.util.concurrent.ThreadPoolExecutor$Worker@df368f8[State = -1, empty > queue]" > "java.util.concurrent.ThreadPoolExecutor$Worker@df368f8[State = -1, empty > queue]": > waiting to lock monitor 0x00007f80356c8800 (object 0x000000033eb70a00, a > java.lang.Object), > which is held by > "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty > queue]" > Java stack information for the threads listed above: > =================================================== > "pool-2-thread-6": > at org.apache.ratis.util.PeerProxyMap.getProxy(PeerProxyMap.java:103) > - waiting to lock <0x000000033eb70a00> (a java.lang.Object) > at > org.apache.ratis.grpc.client.GrpcClientRpc.sendRequestAsyncUnordered(GrpcClientRpc.java:78) > at > org.apache.ratis.client.impl.UnorderedAsync.sendRequestWithRetry(UnorderedAsync.java:75) > at > org.apache.ratis.client.impl.UnorderedAsync.send(UnorderedAsync.java:59) > at > org.apache.ratis.client.impl.RaftClientImpl.sendWatchAsync(RaftClientImpl.java:139) > at > org.apache.hadoop.hdds.scm.XceiverClientRatis.watchForCommit(XceiverClientRatis.java:282) > at > org.apache.hadoop.hdds.scm.storage.CommitWatcher.watchForCommit(CommitWatcher.java:198) > at > org.apache.hadoop.hdds.scm.storage.CommitWatcher.watchOnLastIndex(CommitWatcher.java:161) > at > org.apache.hadoop.hdds.scm.storage.BlockOutputStream.watchForCommit(BlockOutputStream.java:346) > at > org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:482) > at > org.apache.hadoop.hdds.scm.storage.BlockOutputStream.close(BlockOutputStream.java:496) > at > org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.close(BlockOutputStreamEntry.java:143) > at > org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:435) > at > org.apache.hadoop.ozone.client.io.KeyOutputStream.close(KeyOutputStream.java:473) > at > org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:60) > - locked <0x00000003f2ba4240> (a > org.apache.hadoop.ozone.client.io.OzoneOutputStream) > at > org.apache.hadoop.ozone.freon.RandomKeyGenerator.createKey(RandomKeyGenerator.java:710) > at > org.apache.hadoop.ozone.freon.RandomKeyGenerator.access$1100(RandomKeyGenerator.java:88) > at > org.apache.hadoop.ozone.freon.RandomKeyGenerator$ObjectCreator.run(RandomKeyGenerator.java:615) > at > java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.3/Executors.java:515) > at > java.util.concurrent.FutureTask.run(java.base@11.0.3/FutureTask.java:264) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.3/ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.3/ThreadPoolExecutor.java:628) > at java.lang.Thread.run(java.base@11.0.3/Thread.java:834) > "java.util.concurrent.ThreadPoolExecutor$Worker@77329f41[State = -1, empty > queue]": > at > org.apache.ratis.util.SlidingWindow$Client.resetFirstSeqNum(SlidingWindow.java:348) > - waiting to lock <0x000000033eb99b10> (a > org.apache.ratis.util.SlidingWindow$Client) > at > org.apache.ratis.client.impl.OrderedAsync.resetSlidingWindow(OrderedAsync.java:121) > at > org.apache.ratis.client.impl.OrderedAsync$$Lambda$496/0x0000000840498440.accept(Unknown > Source) > at > org.apache.ratis.client.impl.RaftClientImpl.lambda$handleIOException$6(RaftClientImpl.java:349) > at > org.apache.ratis.client.impl.RaftClientImpl$$Lambda$427/0x00000008402e1840.accept(Unknown > Source) > at java.util.Optional.ifPresent(java.base@11.0.3/Optional.java:183) > at > org.apache.ratis.client.impl.RaftClientImpl.handleIOException(RaftClientImpl.java:349) > at > org.apache.ratis.client.impl.OrderedAsync.lambda$sendRequest$10(OrderedAsync.java:236) > at > org.apache.ratis.client.impl.OrderedAsync$$Lambda$358/0x00000008402d7040.apply(Unknown > Source) > at > java.util.concurrent.CompletableFuture.uniExceptionally(java.base@11.0.3/CompletableFuture.java:986) > at > java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(java.base@11.0.3/CompletableFuture.java:970) > at > java.util.concurrent.CompletableFuture.postComplete(java.base@11.0.3/CompletableFuture.java:506) > at > java.util.concurrent.CompletableFuture.completeExceptionally(java.base@11.0.3/CompletableFuture.java:2088) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.completeReplyExceptionally(GrpcClientProtocolClient.java:345) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.close(GrpcClientProtocolClient.java:334) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.access$400(GrpcClientProtocolClient.java:261) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient.lambda$close$1(GrpcClientProtocolClient.java:141) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$$Lambda$435/0x00000008402d1c40.accept(Unknown > Source) > at java.util.Optional.ifPresent(java.base@11.0.3/Optional.java:183) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient.close(GrpcClientProtocolClient.java:141) > at > org.apache.ratis.util.PeerProxyMap$PeerAndProxy.lambda$close$1(PeerProxyMap.java:74) > at > org.apache.ratis.util.PeerProxyMap$PeerAndProxy$$Lambda$430/0x00000008402e1440.run(Unknown > Source) > at > org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$2(LifeCycle.java:231) > at > org.apache.ratis.util.LifeCycle$$Lambda$433/0x00000008402d1040.get(Unknown > Source) > at > org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:251) > at > org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:229) > at > org.apache.ratis.util.PeerProxyMap$PeerAndProxy.close(PeerProxyMap.java:70) > - locked <0x000000008c1ba0e8> (a > org.apache.ratis.util.PeerProxyMap$PeerAndProxy) > at org.apache.ratis.util.PeerProxyMap.resetProxy(PeerProxyMap.java:127) > - locked <0x000000033eb70a00> (a java.lang.Object) > at > org.apache.ratis.util.PeerProxyMap.handleException(PeerProxyMap.java:136) > at > org.apache.ratis.client.impl.RaftClientRpcWithProxy.handleException(RaftClientRpcWithProxy.java:47) > at > org.apache.ratis.client.impl.RaftClientImpl.handleIOException(RaftClientImpl.java:372) > at > org.apache.ratis.client.impl.OrderedAsync.lambda$sendRequest$10(OrderedAsync.java:236) > at > org.apache.ratis.client.impl.OrderedAsync$$Lambda$358/0x00000008402d7040.apply(Unknown > Source) > at > java.util.concurrent.CompletableFuture.uniExceptionally(java.base@11.0.3/CompletableFuture.java:986) > at > java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(java.base@11.0.3/CompletableFuture.java:970) > at > java.util.concurrent.CompletableFuture.postComplete(java.base@11.0.3/CompletableFuture.java:506) > at > java.util.concurrent.CompletableFuture.completeExceptionally(java.base@11.0.3/CompletableFuture.java:2088) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$timeoutCheck$3(GrpcClientProtocolClient.java:324) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers$$Lambda$490/0x000000084049f040.accept(Unknown > Source) > at java.util.Optional.ifPresent(java.base@11.0.3/Optional.java:183) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.handleReplyFuture(GrpcClientProtocolClient.java:329) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.timeoutCheck(GrpcClientProtocolClient.java:324) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$onNext$1(GrpcClientProtocolClient.java:318) > at > org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers$$Lambda$336/0x00000008403b2040.run(Unknown > Source) > at > org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:113) > at > org.apache.ratis.util.TimeoutScheduler$$Lambda$345/0x00000008403ac440.accept(Unknown > Source) > at > org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:133) > at > org.apache.ratis.util.TimeoutScheduler$$Lambda$349/0x00000008403a1440.run(Unknown > Source) > at > org.apache.ratis.util.LogUtils$1$$Lambda$448/0x00000008402d6840.run(Unknown > Source) > at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:50) > at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:91) > at > java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.3/Executors.java:515) > at > java.util.concurrent.FutureTask.run(java.base@11.0.3/FutureTask.java:264) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@11.0.3/ScheduledThreadPoolExecutor.java:304) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.3/ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.3/ThreadPoolExecutor.java:628) > at java.lang.Thread.run(java.base@11.0.3/Thread.java:834) > "java.util.concurrent.ThreadPoolExecutor$Worker@df368f8[State = -1, empty > queue]": > at org.apache.ratis.util.PeerProxyMap.getProxy(PeerProxyMap.java:103) > - waiting to lock <0x000000033eb70a00> (a java.lang.Object) > at > org.apache.ratis.grpc.client.GrpcClientRpc.sendRequestAsync(GrpcClientRpc.java:66) > at > org.apache.ratis.client.impl.OrderedAsync.sendRequest(OrderedAsync.java:208) > at > org.apache.ratis.client.impl.OrderedAsync.sendRequestWithRetry(OrderedAsync.java:169) > at > org.apache.ratis.client.impl.OrderedAsync$$Lambda$449/0x00000008402d6c40.accept(Unknown > Source) > at > org.apache.ratis.util.SlidingWindow$Client.sendOrDelayRequest(SlidingWindow.java:278) > at > org.apache.ratis.util.SlidingWindow$Client.retry(SlidingWindow.java:294) > - locked <0x000000033eb99b10> (a > org.apache.ratis.util.SlidingWindow$Client) > at > org.apache.ratis.client.impl.OrderedAsync.lambda$scheduleWithTimeout$7(OrderedAsync.java:195) > at > org.apache.ratis.client.impl.OrderedAsync$$Lambda$444/0x00000008402d4840.run(Unknown > Source) > at > org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:113) > at > org.apache.ratis.util.TimeoutScheduler$$Lambda$345/0x00000008403ac440.accept(Unknown > Source) > at > org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:133) > at > org.apache.ratis.util.TimeoutScheduler$$Lambda$349/0x00000008403a1440.run(Unknown > Source) > at > org.apache.ratis.util.LogUtils$1$$Lambda$448/0x00000008402d6840.run(Unknown > Source) > at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:50) > at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:91) > at > java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.3/Executors.java:515) > at > java.util.concurrent.FutureTask.run(java.base@11.0.3/FutureTask.java:264) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@11.0.3/ScheduledThreadPoolExecutor.java:304) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.3/ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.3/ThreadPoolExecutor.java:628) > at java.lang.Thread.run(java.base@11.0.3/Thread.java:834) > Found 1 deadlock. > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)