esteban commented on a change in pull request #1858: URL: https://github.com/apache/hbase/pull/1858#discussion_r436155356
########## 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: +1 I don't think the assert should be sitting in a hot path. ---------------------------------------------------------------- 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: us...@infra.apache.org