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/07 08:12:35 UTC

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

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



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
##########
@@ -153,35 +200,30 @@ private void scheduleRelogin(Throwable error) {
       LOG.trace("SASL Provider does not support retries");
       return;
     }
-    synchronized (this) {
-      if (reloginInProgress) {
-        return;
-      }
-      reloginInProgress = true;
-      RELOGIN_EXECUTOR.schedule(new Runnable() {
+    if (reloginInProgress) {
+      return;
+    }
+    reloginInProgress = true;
+    RELOGIN_EXECUTOR.schedule(new Runnable() {

Review comment:
       nit: Would you prefer converting to lambda `() -> `? 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
##########
@@ -73,40 +79,77 @@
   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;
+  // make it volatile so in the isActive method below we do not need to switch to the event loop
+  // thread to access this field.
+  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 static final FastThreadLocal<MutableInt> DEPTH = new FastThreadLocal<MutableInt>() {

Review comment:
       nit: if we want to go till depth of 4, want to use `MutableShort`? 

##########
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:
       This is covered already right?




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