You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/03/14 11:14:32 UTC

[GitHub] [ratis] szetszwo commented on a change in pull request #620: RATIS-1549. If the stream client is abnormally disconnected, reconnec…

szetszwo commented on a change in pull request #620:
URL: https://github.com/apache/ratis/pull/620#discussion_r825835544



##########
File path: ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java
##########
@@ -261,15 +362,27 @@ protected void decode(ChannelHandlerContext context, ByteBuf buf, List<Object> o
       f.completeExceptionally(new IllegalStateException(this + ": Failed to offer a future for " + request));
       return f;
     }
+    final Channel channel = connection.getChannelUninterruptibly();
+    if (channel == null) {
+      f.completeExceptionally(new AlreadyClosedException(this + ": Failed to send " + request));
+      return f;
+    }
     LOG.debug("{}: write {}", this, request);
-    getChannel().writeAndFlush(request);
+    channel.writeAndFlush(request).addListener(future -> {
+      if (!future.isSuccess()) {
+        final IOException e = new IOException(this + ": Failed to send " + request, future.cause());
+        LOG.error("Channel write failed", e);
+        if (!f.isDone()) {

Review comment:
       We don't have to check isDone().  The completeExceptionally API has already taken care of it; see https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html#completeExceptionally-java.lang.Throwable-

##########
File path: ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java
##########
@@ -125,30 +139,109 @@ int size() {
     }
   }
 
+  static class Connection {
+    static final TimeDuration RECONNECT = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
+
+    private final String address;
+    private final WorkerGroupGetter workerGroup;
+    private final Supplier<ChannelInitializer<SocketChannel>> channelInitializerSupplier;
+
+    /** The {@link ChannelFuture} is null when this connection is closed. */
+    private final AtomicReference<ChannelFuture> ref;
+
+    Connection(String address, WorkerGroupGetter workerGroup,
+        Supplier<ChannelInitializer<SocketChannel>> channelInitializerSupplier) {
+      this.address = address;
+      this.workerGroup = workerGroup;
+      this.channelInitializerSupplier = channelInitializerSupplier;
+      this.ref = new AtomicReference<>(connect());
+    }
+
+    Channel getChannelUninterruptibly() {
+      final ChannelFuture future = ref.get();
+      if (future == null) {
+        return null; //closed
+      }
+      final Channel channel = future.syncUninterruptibly().channel();
+      if (channel.isOpen()) {
+        return channel;
+      }
+      return reconnect().syncUninterruptibly().channel();
+    }
+
+    private EventLoopGroup getWorkerGroup() {
+      return workerGroup.get();
+    }
+
+    private ChannelFuture connect() {
+      return new Bootstrap()
+          .group(getWorkerGroup())
+          .channel(NioSocketChannel.class)
+          .handler(channelInitializerSupplier.get())
+          .option(ChannelOption.SO_KEEPALIVE, true)
+          .connect(NetUtils.createSocketAddr(address))
+          .addListener(new ChannelFutureListener() {
+            @Override
+            public void operationComplete(ChannelFuture future) {
+              if (!future.isSuccess()) {
+                scheduleReconnect(this + " failed", future.cause());
+              } else {
+                LOG.trace("{} succeed.", this);
+              }
+            }
+          });
+    }
+
+    void scheduleReconnect(String message, Throwable cause) {
+      LOG.warn("{}: {}; schedule reconnecting to {} in {}", this, message, address, RECONNECT);
+      if (cause != null) {
+        LOG.warn("", cause);
+      }
+      getWorkerGroup().schedule(this::reconnect, RECONNECT.getDuration(), RECONNECT.getUnit());
+    }
+
+    private ChannelFuture reconnect() {
+      final MemoizedSupplier<ChannelFuture> supplier = MemoizedSupplier.valueOf(this::connect);
+      final ChannelFuture previous = ref.getAndUpdate(prev -> prev == null? null: supplier.get());
+      if (previous != null) {
+        previous.channel().close();
+      }
+      return supplier.get();

Review comment:
       We should check supplier.isInitialized().
   ```
         return supplier.isInitialized()? supplier.get(): null;
   ```




-- 
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-unsubscribe@ratis.apache.org

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