OneSizeFitsQuorum commented on code in PR #1285:
URL: https://github.com/apache/ratis/pull/1285#discussion_r2383954825


##########
ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java:
##########
@@ -121,6 +125,17 @@ protected void channelRead0(ChannelHandlerContext ctx,
             future.complete(proto);
           }
         }
+
+        @Override
+        public void exceptionCaught(ChannelHandlerContext ctx, Throwable 
cause) {
+          failOutstandingRequests(new IOException("Caught an exception for the 
connection to " + peer, cause));

Review Comment:
   We could keep the same order with close() in line170-171
   



##########
ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java:
##########
@@ -153,9 +168,14 @@ synchronized CompletableFuture<RaftNettyServerReplyProto> 
pollReply() {
     @Override
     public synchronized void close() {
       client.close();
+      failOutstandingRequests(new AlreadyClosedException("Closing connection 
to " + peer));
+    }
+
+    private void failOutstandingRequests(Throwable cause) {

Review Comment:
   maybe we should synchronized for this method as it now maybe called by 
SimpleChannelInboundHandler, we need to ensure atomicity
   



##########
ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java:
##########
@@ -121,6 +125,17 @@ protected void channelRead0(ChannelHandlerContext ctx,
             future.complete(proto);
           }
         }
+
+        @Override
+        public void exceptionCaught(ChannelHandlerContext ctx, Throwable 
cause) {
+          failOutstandingRequests(new IOException("Caught an exception for the 
connection to " + peer, cause));
+          client.close();
+        }
+
+        @Override
+        public void channelInactive(ChannelHandlerContext ctx) {
+          failOutstandingRequests(new AlreadyClosedException("Channel to " + 
peer + " is inactive."));

Review Comment:
   do we need to call super.channelInactive(ctx) here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@ratis.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to