Lokesh Jain created RATIS-776: --------------------------------- Summary: Handle ResourceUnavailabeException properly in Ratis Server Key: RATIS-776 URL: https://issues.apache.org/jira/browse/RATIS-776 Project: Ratis Issue Type: Bug Components: server Reporter: Lokesh Jain
Ratis leader while processing a client request tries to create a pending request. If it is not able to do so it fails the request with ResourceUnavailableException. But the server keeps processing the other requests from the same client. The resources can be released when the other client requests are processed, resulting in out of order processing of client requests. On failure the server should ideally fail all the client requests which need to be processed. {code:java} 2019-12-10 19:50:31,846 [grpc-default-executor-5] INFO ratis.ContainerStateMachine (ContainerStateMachine.java:preAppendTransaction(311)) - append seqNum:2 WriteChunk 2019-12-10 19:50:31,860 [grpc-default-executor-5] INFO ratis.ContainerStateMachine (ContainerStateMachine.java:preAppendTransaction(311)) - append seqNum:3 WriteChunk Caused by: org.apache.ratis.protocol.exceptions.ResourceUnavailableException: 164293f2-68e3-4851-bc46-4a828bd79ffa@group-03010B1A5718: Failed to acquire a pending write request for RaftClientRequest:client-38E7254A5AF1->164293f2-68e3-4851-bc46-4a828bd79ffa@group-03010B1A5718, cid=3, seq=4, RW, Message:000000b2080612343362...(size=182)Caused by: org.apache.ratis.protocol.exceptions.ResourceUnavailableException: 164293f2-68e3-4851-bc46-4a828bd79ffa@group-03010B1A5718: Failed to acquire a pending write request for RaftClientRequest:client-38E7254A5AF1->164293f2-68e3-4851-bc46-4a828bd79ffa@group-03010B1A5718, cid=3, seq=4, RW, Message:000000b2080612343362...(size=182) at org.apache.ratis.server.impl.RaftServerImpl.appendTransaction(RaftServerImpl.java:514) at org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:589) at org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$7(RaftServerProxy.java:333) at org.apache.ratis.server.impl.RaftServerProxy.lambda$null$5(RaftServerProxy.java:328) at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:109) at org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$6(RaftServerProxy.java:328) at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981) at java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124) at org.apache.ratis.server.impl.RaftServerProxy.submitRequest(RaftServerProxy.java:327) at org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:333) at org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:221) at org.apache.ratis.grpc.client.GrpcClientProtocolService$OrderedRequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:327) at org.apache.ratis.util.SlidingWindow$Server.processRequestsFromHead(SlidingWindow.java:429) at org.apache.ratis.util.SlidingWindow$Server.receivedRequest(SlidingWindow.java:421) at org.apache.ratis.grpc.client.GrpcClientProtocolService$OrderedRequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:346) at org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.onNext(GrpcClientProtocolService.java:241) at org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.onNext(GrpcClientProtocolService.java:168) at org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:251) at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:309) at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:292) at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:779) ... 5 more 2019-12-10 19:50:31,860 [grpc-default-executor-5] INFO ratis.ContainerStateMachine (ContainerStateMachine.java:preAppendTransaction(311)) - append seqNum:5 WriteChunk {code} Further while failing the client request server does not invalidate the retry cache. Any retries from the client are therefore ignored in the server. -- This message was sent by Atlassian Jira (v8.3.4#803005)