anmolnar commented on a change in pull request #4125:
URL: https://github.com/apache/hbase/pull/4125#discussion_r829044839
##########
File path:
hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
##########
@@ -297,11 +295,10 @@ public void operationComplete(ChannelFuture future)
throws Exception {
established(ch);
}
}
- }).channel();
+ }).sync().channel();
Review comment:
HBase shell is not able to connect without this. I'm sure I've seen
errors in RS-Master communication as well, but it works in most cases.
The issue is:
```
ERROR: Failed contacting masters after 1 attempts.
Exceptions:
java.io.IOException: Call to address=Andors-MacBook-Pro.local:16000 failed
on local exception: java.io.IOException:
java.lang.UnsupportedOperationException: unsupported message type: Call
(expected: ByteBuf, FileRegion)
```
My thinking is that `connect()` is having a race with `sendRequest()`,
becaue NettyRpcConnection is trying to establish the connection in lazy way:
doesn't do anything until the first request (`Call`) comes in. Because of this
we have to make sure that the Netty pipeline is in a usable state before
letting sendRequest() to push the Call into and seems like `sync()` is enough.
It waits for the connection to be established before we give back the channel.
This is still not ideal IMHO: we could implement a latch to wait for the SSL
handshake to happen before doing anything else - like ZooKeeper does -, but it
might be overkill in this case given that it already works fine with sync().
##########
File path:
hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
##########
@@ -297,11 +295,10 @@ public void operationComplete(ChannelFuture future)
throws Exception {
established(ch);
}
}
- }).channel();
+ }).sync().channel();
Review comment:
HBase shell is not able to connect without this. I'm sure I've seen
errors in RS-Master communication as well, but it works in most cases.
The issue is:
```
ERROR: Failed contacting masters after 1 attempts.
Exceptions:
java.io.IOException: Call to address=Andors-MacBook-Pro.local:16000 failed
on local exception:
java.io.IOException: java.lang.UnsupportedOperationException: unsupported
message type: Call (expected: ByteBuf, FileRegion)
```
My thinking is that `connect()` is having a race with `sendRequest()`,
becaue NettyRpcConnection is trying to establish the connection in lazy way:
doesn't do anything until the first request (`Call`) comes in. Because of this
we have to make sure that the Netty pipeline is in a usable state before
letting sendRequest() to push the Call into and seems like `sync()` is enough.
It waits for the connection to be established before we give back the channel.
This is still not ideal IMHO: we could implement a latch to wait for the SSL
handshake to happen before doing anything else - like ZooKeeper does -, but it
might be overkill in this case given that it already works fine with sync().
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
##########
@@ -181,4 +204,80 @@ public int getNumOpenConnections() {
// allChannels also contains the server channel, so exclude that from the
count.
return channelsCount > 0 ? channelsCount - 1 : channelsCount;
}
+
+ private synchronized void initSSL(ChannelPipeline p, boolean
supportPlaintext) throws
+ X509Exception {
+ SslContext nettySslContext;
+
+ SSLContextAndOptions sslContextAndOptions =
x509Util.getDefaultSSLContextAndOptions();
+ nettySslContext = sslContextAndOptions
+ .createNettyJdkSslContext(sslContextAndOptions.getSSLContext(), false);
+
+ if (supportPlaintext) {
+ p.addLast("ssl", new NettyRpcServer.DualModeSslHandler(nettySslContext));
+ LOG.debug("Dual mode SSL handler added for channel: {}", p.channel());
+ } else {
+ p.addLast("ssl", nettySslContext.newHandler(p.channel().alloc()));
+ LOG.debug("SSL handler added for channel: {}", p.channel());
+ }
+ }
+
+ /**
+ * A handler that detects whether the client would like to use
+ * TLS or not and responds in kind. The first bytes are examined
+ * for the static TLS headers to make the determination and
+ * placed back in the stream with the correct ChannelHandler
+ * instantiated.
+ */
+ class DualModeSslHandler extends OptionalSslHandler {
+
+ DualModeSslHandler(SslContext sslContext) {
+ super(sslContext);
+ }
+
+ @Override
+ protected void decode(ChannelHandlerContext context, ByteBuf in,
List<Object> out)
+ throws Exception {
+ if (in.readableBytes() >= 5) {
+ super.decode(context, in, out);
+ } else if (in.readableBytes() > 0) {
+ // It requires 5 bytes to detect a proper ssl connection. In the
Review comment:
That's copied from the base class.
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
##########
@@ -181,4 +204,80 @@ public int getNumOpenConnections() {
// allChannels also contains the server channel, so exclude that from the
count.
return channelsCount > 0 ? channelsCount - 1 : channelsCount;
}
+
+ private synchronized void initSSL(ChannelPipeline p, boolean
supportPlaintext) throws
+ X509Exception {
+ SslContext nettySslContext;
+
+ SSLContextAndOptions sslContextAndOptions =
x509Util.getDefaultSSLContextAndOptions();
+ nettySslContext = sslContextAndOptions
+ .createNettyJdkSslContext(sslContextAndOptions.getSSLContext(), false);
+
+ if (supportPlaintext) {
+ p.addLast("ssl", new NettyRpcServer.DualModeSslHandler(nettySslContext));
+ LOG.debug("Dual mode SSL handler added for channel: {}", p.channel());
+ } else {
+ p.addLast("ssl", nettySslContext.newHandler(p.channel().alloc()));
+ LOG.debug("SSL handler added for channel: {}", p.channel());
+ }
+ }
+
+ /**
+ * A handler that detects whether the client would like to use
+ * TLS or not and responds in kind. The first bytes are examined
+ * for the static TLS headers to make the determination and
+ * placed back in the stream with the correct ChannelHandler
+ * instantiated.
+ */
+ class DualModeSslHandler extends OptionalSslHandler {
+
+ DualModeSslHandler(SslContext sslContext) {
+ super(sslContext);
+ }
+
+ @Override
+ protected void decode(ChannelHandlerContext context, ByteBuf in,
List<Object> out)
+ throws Exception {
+ if (in.readableBytes() >= 5) {
+ super.decode(context, in, out);
+ } else if (in.readableBytes() > 0) {
+ // It requires 5 bytes to detect a proper ssl connection. In the
+ // case that the server receives fewer, check if we can fail to
plaintext.
+ // This will occur when for any four letter work commands.
+ if (TLS_HANDSHAKE_RECORD_TYPE != in.getByte(0)) {
+ LOG.debug("first byte {} does not match TLS handshake, failing to
plaintext",
+ in.getByte(0));
+ handleNonSsl(context);
Review comment:
I ended up removing this class. That's something which was needed for
ZooKeeper 4LW commands which is obviously not the case for HBase.
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
##########
@@ -181,4 +204,80 @@ public int getNumOpenConnections() {
// allChannels also contains the server channel, so exclude that from the
count.
return channelsCount > 0 ? channelsCount - 1 : channelsCount;
}
+
+ private synchronized void initSSL(ChannelPipeline p, boolean
supportPlaintext) throws
+ X509Exception {
+ SslContext nettySslContext;
+
+ SSLContextAndOptions sslContextAndOptions =
x509Util.getDefaultSSLContextAndOptions();
+ nettySslContext = sslContextAndOptions
+ .createNettyJdkSslContext(sslContextAndOptions.getSSLContext(), false);
+
+ if (supportPlaintext) {
Review comment:
Good point. Yes, we should support it. Let me add a config option.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]