bharathv commented on a change in pull request #1858:
URL: https://github.com/apache/hbase/pull/1858#discussion_r436321731
##########
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:
Thanks for the explanation.
> The lock is to make sure that we always use the up to date Channel
instance.
Ya, this is the part that was not clear to me. As I understand it, Netty
channel implementations are thread-safe (correct me if I'm wrong here). If I
read the netty code correctly, all the channel/pipeline operations are
internally enqueued on the eventloop.
So why can't we just make it a volatile and we have the up to date channel
for all threads to operate on? That removes one lock from the nested locking
protocol for Channels and RpcControllers. In other words, whats the need to
serialize all the operations on a channel inside NettyRpcConnection, before
with synchronized and now with the event loop.
(Also posted a question for you on jira, just so that I understand the
problem 100%, thanks for your patience).
##########
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:
Hmm okay. I still think the locking can be made fine grained with rw
locks for channel and totally get rid of eventloop but its okay, your perf run
shows no regression, so its fine.
##########
File path:
hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcConnection.java
##########
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.ipc;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.net.InetSocketAddress;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.security.User;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+public class TestNettyRpcConnection {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestNettyRpcConnection.class);
+
+ private static final Logger LOG =
LoggerFactory.getLogger(TestNettyRpcConnection.class);
+
+ private static NettyRpcClient CLIENT;
+
+ private static NettyRpcConnection CONN;
+
+ @BeforeClass
+ public static void setUp() throws IOException {
+ CLIENT = new NettyRpcClient(HBaseConfiguration.create());
+ CONN = new NettyRpcConnection(CLIENT,
+ new ConnectionId(User.getCurrent(), "test", new
InetSocketAddress("localhost", 1234)));
+ }
+
+ @AfterClass
+ public static void tearDown() throws IOException {
+ Closeables.close(CLIENT, true);
+ }
+
+ @Test
+ public void testPrivateMethodExecutedInEventLoop() throws
IllegalAccessException {
+ // make sure the test is executed with "-ea"
+ assertThrows(AssertionError.class, () -> {
+ assert false;
+ });
+ for (Method method : NettyRpcConnection.class.getDeclaredMethods()) {
+ if (Modifier.isPrivate(method.getModifiers()) &&
!method.getName().contains("$")) {
+ LOG.info("checking {}", method);
+ method.setAccessible(true);
+ // all private methods should be called inside the event loop thread,
so calling it from
Review comment:
haha, nice test :-)
----------------------------------------------------------------
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:
[email protected]