You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/06/05 19:50:24 UTC

[GitHub] [hbase] bharathv commented on a change in pull request #1858: HBASE-24506 async client deadlock

bharathv commented on a change in pull request #1858:
URL: https://github.com/apache/hbase/pull/1858#discussion_r436123009



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
##########
@@ -73,40 +73,53 @@
   private static final Logger LOG = LoggerFactory.getLogger(NettyRpcConnection.class);
 
   private static final ScheduledExecutorService RELOGIN_EXECUTOR =
-      Executors.newSingleThreadScheduledExecutor(Threads.newDaemonThreadFactory("Relogin"));
+    Executors.newSingleThreadScheduledExecutor(Threads.newDaemonThreadFactory("Relogin"));
 
   private final NettyRpcClient rpcClient;
 
+  // the event loop used to set up the connection, we will also execute other operations for this
+  // connection in this event loop, to avoid locking everywhere.
+  private final EventLoop eventLoop;
+
   private ByteBuf connectionHeaderPreamble;
 
   private ByteBuf connectionHeaderWithLength;
 
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
-      justification = "connect is also under lock as notifyOnCancel will call our action directly")
-  private Channel channel;
+  private volatile Channel channel;
 
   NettyRpcConnection(NettyRpcClient rpcClient, ConnectionId remoteId) throws IOException {
     super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId,
-        rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor);
+      rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor);
     this.rpcClient = rpcClient;
+    this.eventLoop = rpcClient.group.next();
     byte[] connectionHeaderPreamble = getConnectionHeaderPreamble();
     this.connectionHeaderPreamble =
-        Unpooled.directBuffer(connectionHeaderPreamble.length).writeBytes(connectionHeaderPreamble);
+      Unpooled.directBuffer(connectionHeaderPreamble.length).writeBytes(connectionHeaderPreamble);
     ConnectionHeader header = getConnectionHeader();
     this.connectionHeaderWithLength = Unpooled.directBuffer(4 + header.getSerializedSize());
     this.connectionHeaderWithLength.writeInt(header.getSerializedSize());
     header.writeTo(new ByteBufOutputStream(this.connectionHeaderWithLength));
   }
 
-  @Override
-  protected synchronized void callTimeout(Call call) {
-    if (channel != null) {
-      channel.pipeline().fireUserEventTriggered(new CallEvent(TIMEOUT, call));
+  private void execute(Runnable action) {
+    if (eventLoop.inEventLoop()) {

Review comment:
       nit: would the first if() be ever true for client threads?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
##########
@@ -253,52 +268,38 @@ public void operationComplete(Future<Boolean> future) throws Exception {
   }
 
   private void connect() {
+    assert eventLoop.inEventLoop();
     LOG.trace("Connecting to {}", remoteId.address);
 
-    this.channel = new Bootstrap().group(rpcClient.group).channel(rpcClient.channelClass)
-        .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())
-        .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive)
-        .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO)
-        .handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr)
-        .remoteAddress(remoteId.address).connect().addListener(new ChannelFutureListener() {
-
-          @Override
-          public void operationComplete(ChannelFuture future) throws Exception {
-            Channel ch = future.channel();
-            if (!future.isSuccess()) {
-              failInit(ch, toIOE(future.cause()));
-              rpcClient.failedServers.addToFailedServers(remoteId.address, future.cause());
-              return;
-            }
-            ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate());
-            if (useSasl) {
-              saslNegotiate(ch);
-            } else {
-              // send the connection header to server
-              ch.write(connectionHeaderWithLength.retainedDuplicate());
-              established(ch);
-            }
-          }
-        }).channel();
-  }
+    this.channel = new Bootstrap().group(eventLoop).channel(rpcClient.channelClass)
+      .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())
+      .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive)
+      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO)
+      .handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr)
+      .remoteAddress(remoteId.address).connect().addListener(new ChannelFutureListener() {
 
-  private void write(Channel ch, final Call call) {
-    ch.writeAndFlush(call).addListener(new ChannelFutureListener() {
-
-      @Override
-      public void operationComplete(ChannelFuture future) throws Exception {
-        // Fail the call if we failed to write it out. This usually because the channel is
-        // closed. This is needed because we may shutdown the channel inside event loop and
-        // there may still be some pending calls in the event loop queue after us.
-        if (!future.isSuccess()) {
-          call.setException(toIOE(future.cause()));
+        @Override
+        public void operationComplete(ChannelFuture future) throws Exception {
+          Channel ch = future.channel();
+          if (!future.isSuccess()) {
+            failInit(ch, toIOE(future.cause()));
+            rpcClient.failedServers.addToFailedServers(remoteId.address, future.cause());
+            return;
+          }
+          ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate());
+          if (useSasl) {
+            saslNegotiate(ch);
+          } else {
+            // send the connection header to server
+            ch.write(connectionHeaderWithLength.retainedDuplicate());
+            established(ch);
+          }
         }
-      }
-    });
+      }).channel();
   }
 
-  @Override
-  public synchronized void sendRequest(final Call call, HBaseRpcController hrc) throws IOException {

Review comment:
       Just a question, why does it have to be giant synchronized? What is the un-safe shared state? Wondering if we can narrow down the lock and simplify the threading model to fix the actual problem.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
##########
@@ -253,52 +268,38 @@ public void operationComplete(Future<Boolean> future) throws Exception {
   }
 
   private void connect() {
+    assert eventLoop.inEventLoop();
     LOG.trace("Connecting to {}", remoteId.address);
 
-    this.channel = new Bootstrap().group(rpcClient.group).channel(rpcClient.channelClass)
-        .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())
-        .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive)
-        .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO)
-        .handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr)
-        .remoteAddress(remoteId.address).connect().addListener(new ChannelFutureListener() {
-
-          @Override
-          public void operationComplete(ChannelFuture future) throws Exception {
-            Channel ch = future.channel();
-            if (!future.isSuccess()) {
-              failInit(ch, toIOE(future.cause()));
-              rpcClient.failedServers.addToFailedServers(remoteId.address, future.cause());
-              return;
-            }
-            ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate());
-            if (useSasl) {
-              saslNegotiate(ch);
-            } else {
-              // send the connection header to server
-              ch.write(connectionHeaderWithLength.retainedDuplicate());
-              established(ch);
-            }
-          }
-        }).channel();
-  }
+    this.channel = new Bootstrap().group(eventLoop).channel(rpcClient.channelClass)
+      .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())
+      .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive)
+      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO)
+      .handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr)
+      .remoteAddress(remoteId.address).connect().addListener(new ChannelFutureListener() {
 
-  private void write(Channel ch, final Call call) {
-    ch.writeAndFlush(call).addListener(new ChannelFutureListener() {
-
-      @Override
-      public void operationComplete(ChannelFuture future) throws Exception {
-        // Fail the call if we failed to write it out. This usually because the channel is
-        // closed. This is needed because we may shutdown the channel inside event loop and
-        // there may still be some pending calls in the event loop queue after us.
-        if (!future.isSuccess()) {
-          call.setException(toIOE(future.cause()));
+        @Override
+        public void operationComplete(ChannelFuture future) throws Exception {
+          Channel ch = future.channel();
+          if (!future.isSuccess()) {
+            failInit(ch, toIOE(future.cause()));
+            rpcClient.failedServers.addToFailedServers(remoteId.address, future.cause());
+            return;
+          }
+          ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate());
+          if (useSasl) {
+            saslNegotiate(ch);
+          } else {
+            // send the connection header to server
+            ch.write(connectionHeaderWithLength.retainedDuplicate());
+            established(ch);
+          }
         }
-      }
-    });
+      }).channel();
   }
 
-  @Override
-  public synchronized void sendRequest(final Call call, HBaseRpcController hrc) throws IOException {
+  private void sendRequest0(Call call, HBaseRpcController hrc) throws IOException {
+    assert eventLoop.inEventLoop();

Review comment:
       Curious to know your thoughts on the performance implications of this change. Now that the event loop is responsible for more than what it was doing before, how would it affect the overall throughput.




----------------------------------------------------------------
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.

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