diff --git a/bin/hbase b/bin/hbase index 4a17c810594f..d34fdffd15c5 100755 --- a/bin/hbase +++ b/bin/hbase @@ -107,6 +107,7 @@ show_usage() { echo " cellcounter Run CellCounter tool" echo " pre-upgrade Run Pre-Upgrade validator tool" echo " hbtop Run HBTop tool" + echo " credential Run the Hadoop Credential Shell" echo " CLASSNAME Run the class named CLASSNAME" } @@ -759,6 +760,8 @@ elif [ "$COMMAND" = "hbtop" ] ; then HBASE_HBTOP_OPTS="${HBASE_HBTOP_OPTS} -Dlog4j2.configurationFile=file:${HBASE_HOME}/conf/log4j2-hbtop.properties" fi HBASE_OPTS="${HBASE_OPTS} ${HBASE_HBTOP_OPTS}" +elif [ "$COMMAND" = "credential" ] ; then + CLASS='org.apache.hadoop.security.alias.CredentialShell' else CLASS=$COMMAND if [[ "$CLASS" =~ .*IntegrationTest.* ]] ; then diff --git a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java index af5154b94608..48656329df2e 100644 --- a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java +++ b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java @@ -188,4 +188,13 @@ public static UserGroupInformation loginAndReturnUGI(Configuration conf, String UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keyTabFileLocation); return ugi; } + + public static UserGroupInformation loginKerberosPrincipal(String krbKeytab, String krbPrincipal) + throws Exception { + Configuration conf = new Configuration(); + conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab); + return UserGroupInformation.getLoginUser(); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java index 630261a09273..0afd07d79513 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java @@ -230,7 +230,7 @@ public static String getDefaultCodec(final Configuration c) { * Encapsulate the ugly casting and RuntimeException conversion in private method. * @return Codec to use on this client. */ - Codec getCodec() { + protected Codec getCodec() { // For NO CODEC, "hbase.client.rpc.codec" must be configured with empty string AND // "hbase.client.default.rpc.codec" also -- because default is to do cell block encoding. String className = conf.get(HConstants.RPC_CODEC_CONF_KEY, getDefaultCodec(this.conf)); @@ -251,7 +251,7 @@ public boolean hasCellBlockSupport() { } // for writing tests that want to throw exception when connecting. - boolean isTcpNoDelay() { + protected boolean isTcpNoDelay() { return tcpNoDelay; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java index 22123eb2a600..606749315f41 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java @@ -19,10 +19,14 @@ import java.io.IOException; import java.net.SocketAddress; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.MetricsConnection; +import org.apache.hadoop.hbase.exceptions.X509Exception; +import org.apache.hadoop.hbase.io.FileChangeWatcher; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; import org.apache.hadoop.hbase.util.NettyFutureUtils; import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; @@ -31,6 +35,7 @@ import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory; /** @@ -45,6 +50,9 @@ public class NettyRpcClient extends AbstractRpcClient { final Class channelClass; private final boolean shutdownGroupWhenClose; + private final AtomicReference sslContextForClient = new AtomicReference<>(); + private final AtomicReference keyStoreWatcher = new AtomicReference<>(); + private final AtomicReference trustStoreWatcher = new AtomicReference<>(); public NettyRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress, MetricsConnection metrics) { @@ -67,7 +75,7 @@ public NettyRpcClient(Configuration configuration, String clusterId, SocketAddre } /** Used in test only. */ - NettyRpcClient(Configuration configuration) { + public NettyRpcClient(Configuration configuration) { this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null); } @@ -81,5 +89,31 @@ protected void closeInternal() { if (shutdownGroupWhenClose) { NettyFutureUtils.consume(group.shutdownGracefully()); } + FileChangeWatcher ks = keyStoreWatcher.getAndSet(null); + if (ks != null) { + ks.stop(); + } + FileChangeWatcher ts = trustStoreWatcher.getAndSet(null); + if (ts != null) { + ts.stop(); + } + } + + SslContext getSslContext() throws X509Exception, IOException { + SslContext result = sslContextForClient.get(); + if (result == null) { + result = X509Util.createSslContextForClient(conf); + if (!sslContextForClient.compareAndSet(null, result)) { + // lost the race, another thread already set the value + result = sslContextForClient.get(); + } else if ( + keyStoreWatcher.get() == null && trustStoreWatcher.get() == null + && conf.getBoolean(X509Util.TLS_CERT_RELOAD, false) + ) { + X509Util.enableCertFileReloading(conf, keyStoreWatcher, trustStoreWatcher, + () -> sslContextForClient.set(null)); + } + } + return result; } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java index 701aad0add5f..ff929d970340 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java @@ -30,6 +30,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; import org.apache.hadoop.hbase.ipc.BufferCallBeforeInitHandler.BufferCallEvent; import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback; import org.apache.hadoop.hbase.security.NettyHBaseRpcConnectionHeaderHandler; @@ -56,6 +57,8 @@ import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline; import org.apache.hbase.thirdparty.io.netty.channel.EventLoop; import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler; import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler; import org.apache.hbase.thirdparty.io.netty.handler.timeout.ReadTimeoutHandler; import org.apache.hbase.thirdparty.io.netty.util.ReferenceCountUtil; @@ -219,16 +222,14 @@ private void saslNegotiate(final Channel ch) { return; } ch.pipeline().addBefore(BufferCallBeforeInitHandler.NAME, null, new SaslChallengeDecoder()) - .addBefore(BufferCallBeforeInitHandler.NAME, null, saslHandler); + .addBefore(BufferCallBeforeInitHandler.NAME, NettyHBaseSaslRpcClientHandler.HANDLER_NAME, + saslHandler); NettyFutureUtils.addListener(saslPromise, new FutureListener() { @Override public void operationComplete(Future future) throws Exception { if (future.isSuccess()) { ChannelPipeline p = ch.pipeline(); - p.remove(SaslChallengeDecoder.class); - p.remove(NettyHBaseSaslRpcClientHandler.class); - // check if negotiate with server for connection header is necessary if (saslHandler.isNeedProcessConnectionHeader()) { Promise connectionHeaderPromise = ch.eventLoop().newPromise(); @@ -250,7 +251,7 @@ public void operationComplete(Future future) throws Exception { ChannelPipeline p = ch.pipeline(); p.remove(readTimeoutHandlerName); p.remove(NettyHBaseRpcConnectionHeaderHandler.class); - // don't send connection header, NettyHBaseRpcConnectionHeaderHandler + // don't send connection header, NettyHbaseRpcConnectionHeaderHandler // sent it already established(ch); } else { @@ -283,26 +284,26 @@ private void connect() throws UnknownHostException { .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive) .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO) .handler(new ChannelInitializer() { - @Override protected void initChannel(Channel ch) throws Exception { + if (conf.getBoolean(X509Util.HBASE_CLIENT_NETTY_TLS_ENABLED, false)) { + SslContext sslContext = rpcClient.getSslContext(); + SslHandler sslHandler = sslContext.newHandler(ch.alloc(), + remoteId.address.getHostName(), remoteId.address.getPort()); + sslHandler.setHandshakeTimeoutMillis( + conf.getInt(X509Util.HBASE_CLIENT_NETTY_TLS_HANDSHAKETIMEOUT, + X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS)); + ch.pipeline().addFirst(sslHandler); + LOG.info("SSL handler added with handshake timeout {} ms", + sslHandler.getHandshakeTimeoutMillis()); + } ch.pipeline().addLast(BufferCallBeforeInitHandler.NAME, new BufferCallBeforeInitHandler()); } }).localAddress(rpcClient.localAddr).remoteAddress(remoteAddr).connect() .addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - Channel ch = future.channel(); - if (!future.isSuccess()) { - IOException ex = toIOE(future.cause()); - LOG.warn( - "Exception encountered while connecting to the server " + remoteId.getAddress(), ex); - failInit(ch, ex); - rpcClient.failedServers.addToFailedServers(remoteId.getAddress(), future.cause()); - return; - } + private void succeed(Channel ch) throws IOException { NettyFutureUtils.safeWriteAndFlush(ch, connectionHeaderPreamble.retainedDuplicate()); if (useSasl) { saslNegotiate(ch); @@ -312,6 +313,35 @@ public void operationComplete(ChannelFuture future) throws Exception { established(ch); } } + + private void fail(Channel ch, Throwable error) { + IOException ex = toIOE(error); + LOG.warn("Exception encountered while connecting to the server " + remoteId.getAddress(), + ex); + failInit(ch, ex); + rpcClient.failedServers.addToFailedServers(remoteId.getAddress(), error); + } + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + Channel ch = future.channel(); + if (!future.isSuccess()) { + fail(ch, future.cause()); + return; + } + SslHandler sslHandler = ch.pipeline().get(SslHandler.class); + if (sslHandler != null) { + NettyFutureUtils.addListener(sslHandler.handshakeFuture(), f -> { + if (f.isSuccess()) { + succeed(ch); + } else { + fail(ch, f.cause()); + } + }); + } else { + succeed(ch); + } + } }).channel(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java deleted file mode 100644 index 31ed191f91a6..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.security; - -import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; -import org.apache.yetus.audience.InterfaceAudience; - -import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; -import org.apache.hbase.thirdparty.io.netty.buffer.Unpooled; -import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; -import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; - -/** - * Unwrap messages with Crypto AES. Should be placed after a - * io.netty.handler.codec.LengthFieldBasedFrameDecoder - */ -@InterfaceAudience.Private -public class CryptoAESUnwrapHandler extends SimpleChannelInboundHandler { - - private final CryptoAES cryptoAES; - - public CryptoAESUnwrapHandler(CryptoAES cryptoAES) { - this.cryptoAES = cryptoAES; - } - - @Override - protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { - byte[] bytes = new byte[msg.readableBytes()]; - msg.readBytes(bytes); - ctx.fireChannelRead(Unpooled.wrappedBuffer(cryptoAES.unwrap(bytes, 0, bytes.length))); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java deleted file mode 100644 index a99d097ff2d8..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.security; - -import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; -import org.apache.yetus.audience.InterfaceAudience; - -import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; -import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; -import org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToByteEncoder; - -/** - * wrap messages with Crypto AES. - */ -@InterfaceAudience.Private -public class CryptoAESWrapHandler extends MessageToByteEncoder { - - private final CryptoAES cryptoAES; - - public CryptoAESWrapHandler(CryptoAES cryptoAES) { - this.cryptoAES = cryptoAES; - } - - @Override - protected void encode(ChannelHandlerContext ctx, ByteBuf msg, ByteBuf out) throws Exception { - byte[] bytes = new byte[msg.readableBytes()]; - msg.readBytes(bytes); - byte[] wrapperBytes = cryptoAES.wrap(bytes, 0, bytes.length); - out.ensureWritable(4 + wrapperBytes.length); - out.writeInt(wrapperBytes.length); - out.writeBytes(wrapperBytes); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java index 0a97e922e78d..cb680b517244 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java @@ -26,7 +26,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline; import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; -import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder; import org.apache.hbase.thirdparty.io.netty.util.concurrent.Promise; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; @@ -92,10 +91,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { * Remove handlers for sasl encryption and add handlers for Crypto AES encryption */ private void setupCryptoAESHandler(ChannelPipeline p, CryptoAES cryptoAES) { - p.remove(SaslWrapHandler.class); - p.remove(SaslUnwrapHandler.class); - String lengthDecoder = p.context(LengthFieldBasedFrameDecoder.class).name(); - p.addAfter(lengthDecoder, null, new CryptoAESUnwrapHandler(cryptoAES)); - p.addAfter(lengthDecoder, null, new CryptoAESWrapHandler(cryptoAES)); + p.replace(SaslWrapHandler.class, null, new SaslWrapHandler(cryptoAES::wrap)); + p.replace(SaslUnwrapHandler.class, null, new SaslUnwrapHandler(cryptoAES::unwrap)); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java index 9b16a41afe4e..fe5481a10b25 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java @@ -45,16 +45,16 @@ public NettyHBaseSaslRpcClient(Configuration conf, SaslClientAuthenticationProvi super(conf, provider, token, serverAddr, securityInfo, fallbackAllowed, rpcProtection); } - public void setupSaslHandler(ChannelPipeline p) { + public void setupSaslHandler(ChannelPipeline p, String addAfter) { String qop = (String) saslClient.getNegotiatedProperty(Sasl.QOP); LOG.trace("SASL client context established. Negotiated QoP {}", qop); if (qop == null || "auth".equalsIgnoreCase(qop)) { return; } // add wrap and unwrap handlers to pipeline. - p.addFirst(new SaslWrapHandler(saslClient), - new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4), - new SaslUnwrapHandler(saslClient)); + p.addAfter(addAfter, null, new SaslUnwrapHandler(saslClient::unwrap)) + .addAfter(addAfter, null, new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)) + .addAfter(addAfter, null, new SaslWrapHandler(saslClient::wrap)); } public String getSaslQOP() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java index 525a78d0ae8d..32856da18b50 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.exceptions.ConnectionClosedException; import org.apache.hadoop.hbase.ipc.FallbackDisallowedException; import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProvider; +import org.apache.hadoop.hbase.util.NettyFutureUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; @@ -33,6 +34,7 @@ import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline; import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; import org.apache.hbase.thirdparty.io.netty.util.concurrent.Promise; @@ -45,6 +47,8 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler< private static final Logger LOG = LoggerFactory.getLogger(NettyHBaseSaslRpcClientHandler.class); + public static final String HANDLER_NAME = "SaslRpcClientHandler"; + private final Promise saslPromise; private final UserGroupInformation ugi; @@ -74,7 +78,7 @@ public NettyHBaseSaslRpcClientHandler(Promise saslPromise, UserGroupInf private void writeResponse(ChannelHandlerContext ctx, byte[] response) { LOG.trace("Sending token size={} from initSASLContext.", response.length); - ctx.writeAndFlush( + NettyFutureUtils.safeWriteAndFlush(ctx, ctx.alloc().buffer(4 + response.length).writeInt(response.length).writeBytes(response)); } @@ -83,7 +87,11 @@ private void tryComplete(ChannelHandlerContext ctx) { return; } - saslRpcClient.setupSaslHandler(ctx.pipeline()); + ChannelPipeline p = ctx.pipeline(); + saslRpcClient.setupSaslHandler(p, HANDLER_NAME); + p.remove(SaslChallengeDecoder.class); + p.remove(this); + setCryptoAESOption(); saslPromise.setSuccess(true); @@ -102,6 +110,9 @@ public boolean isNeedProcessConnectionHeader() { @Override public void handlerAdded(ChannelHandlerContext ctx) { + // dispose the saslRpcClient when the channel is closed, since saslRpcClient is final, it is + // safe to reference it in lambda expr. + NettyFutureUtils.addListener(ctx.channel().closeFuture(), f -> saslRpcClient.dispose()); try { byte[] initialResponse = ugi.doAs(new PrivilegedExceptionAction() { @@ -151,14 +162,12 @@ public byte[] run() throws Exception { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { - saslRpcClient.dispose(); saslPromise.tryFailure(new ConnectionClosedException("Connection closed")); ctx.fireChannelInactive(); } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - saslRpcClient.dispose(); saslPromise.tryFailure(cause); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java index dfc36e4ba314..87e518dae4aa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.security; -import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; @@ -32,22 +32,20 @@ @InterfaceAudience.Private public class SaslUnwrapHandler extends SimpleChannelInboundHandler { - private final SaslClient saslClient; - - public SaslUnwrapHandler(SaslClient saslClient) { - this.saslClient = saslClient; + public interface Unwrapper { + byte[] unwrap(byte[] incoming, int offset, int len) throws SaslException; } - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - SaslUtil.safeDispose(saslClient); - ctx.fireChannelInactive(); + private final Unwrapper unwrapper; + + public SaslUnwrapHandler(Unwrapper unwrapper) { + this.unwrapper = unwrapper; } @Override protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { byte[] bytes = new byte[msg.readableBytes()]; msg.readBytes(bytes); - ctx.fireChannelRead(Unpooled.wrappedBuffer(saslClient.unwrap(bytes, 0, bytes.length))); + ctx.fireChannelRead(Unpooled.wrappedBuffer(unwrapper.unwrap(bytes, 0, bytes.length))); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java index 21f70e3f1e4c..6caf2a3e8f51 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.security; -import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; @@ -30,17 +30,21 @@ @InterfaceAudience.Private public class SaslWrapHandler extends MessageToByteEncoder { - private final SaslClient saslClient; + public interface Wrapper { + byte[] wrap(byte[] outgoing, int offset, int len) throws SaslException; + } + + private final Wrapper wrapper; - public SaslWrapHandler(SaslClient saslClient) { - this.saslClient = saslClient; + public SaslWrapHandler(Wrapper wrapper) { + this.wrapper = wrapper; } @Override protected void encode(ChannelHandlerContext ctx, ByteBuf msg, ByteBuf out) throws Exception { byte[] bytes = new byte[msg.readableBytes()]; msg.readBytes(bytes); - byte[] wrapperBytes = saslClient.wrap(bytes, 0, bytes.length); + byte[] wrapperBytes = wrapper.wrap(bytes, 0, bytes.length); out.ensureWritable(4 + wrapperBytes.length); out.writeInt(wrapperBytes.length); out.writeBytes(wrapperBytes); diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index e9c07b80cf0e..ff5daf7c1f7c 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -154,6 +154,16 @@ kerb-simplekdc test + + org.bouncycastle + bcprov-jdk15on + test + + + org.bouncycastle + bcpkix-jdk15on + test + diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/KeyManagerException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/KeyManagerException.java new file mode 100644 index 000000000000..ae337d05fe3e --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/KeyManagerException.java @@ -0,0 +1,33 @@ +/* + * 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.exceptions; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class KeyManagerException extends X509Exception { + + public KeyManagerException(String message) { + super(message); + } + + public KeyManagerException(Throwable cause) { + super(cause); + } + +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SSLContextException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SSLContextException.java new file mode 100644 index 000000000000..0999a6f9bd51 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SSLContextException.java @@ -0,0 +1,37 @@ +/* + * 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.exceptions; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class SSLContextException extends X509Exception { + + public SSLContextException(String message) { + super(message); + } + + public SSLContextException(Throwable cause) { + super(cause); + } + + public SSLContextException(String message, Throwable cause) { + super(message, cause); + } + +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/TrustManagerException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/TrustManagerException.java new file mode 100644 index 000000000000..131ad7d2cd39 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/TrustManagerException.java @@ -0,0 +1,33 @@ +/* + * 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.exceptions; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class TrustManagerException extends X509Exception { + + public TrustManagerException(String message) { + super(message); + } + + public TrustManagerException(Throwable cause) { + super(cause); + } + +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/X509Exception.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/X509Exception.java new file mode 100644 index 000000000000..10bed79351d2 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/X509Exception.java @@ -0,0 +1,43 @@ +/* + * 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.exceptions; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +public class X509Exception extends HBaseException { + + public X509Exception(String message) { + super(message); + } + + public X509Exception(Throwable cause) { + super(cause); + } + + public X509Exception(String message, Throwable cause) { + super(message, cause); + } + +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/FileChangeWatcher.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/FileChangeWatcher.java new file mode 100644 index 000000000000..77e0e4e750ce --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/FileChangeWatcher.java @@ -0,0 +1,251 @@ +/* + * 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.io; + +import java.io.IOException; +import java.nio.file.ClosedWatchServiceException; +import java.nio.file.FileSystem; +import java.nio.file.Path; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.function.Consumer; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.server.ZooKeeperThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Instances of this class can be used to watch a directory for file changes. When a file is added + * to, deleted from, or is modified in the given directory, the callback provided by the user will + * be called from a background thread. Some things to keep in mind: + *
    + *
  • The callback should be thread-safe.
  • + *
  • Changes that happen around the time the thread is started may be missed.
  • + *
  • There is a delay between a file changing and the callback firing.
  • + *
  • The watch is not recursive - changes to subdirectories will not trigger a callback.
  • + *
+ *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +public final class FileChangeWatcher { + + private static final Logger LOG = LoggerFactory.getLogger(FileChangeWatcher.class); + + public enum State { + NEW, // object created but start() not called yet + STARTING, // start() called but background thread has not entered main loop + RUNNING, // background thread is running + STOPPING, // stop() called but background thread has not exited main loop + STOPPED // stop() called and background thread has exited, or background thread crashed + } + + private final WatcherThread watcherThread; + private State state; // protected by synchronized(this) + + /** + * Creates a watcher that watches dirPath and invokes callback on + * changes. + * @param dirPath the directory to watch. + * @param callback the callback to invoke with events. event.kind() will return the + * type of event, and event.context() will return the filename + * relative to dirPath. + * @throws IOException if there is an error creating the WatchService. + */ + public FileChangeWatcher(Path dirPath, Consumer> callback) throws IOException { + FileSystem fs = dirPath.getFileSystem(); + WatchService watchService = fs.newWatchService(); + + LOG.debug("Registering with watch service: {}", dirPath); + + dirPath.register(watchService, + new WatchEvent.Kind[] { StandardWatchEventKinds.ENTRY_CREATE, + StandardWatchEventKinds.ENTRY_DELETE, StandardWatchEventKinds.ENTRY_MODIFY, + StandardWatchEventKinds.OVERFLOW }); + state = State.NEW; + this.watcherThread = new WatcherThread(watchService, callback); + this.watcherThread.setDaemon(true); + } + + /** + * Returns the current {@link FileChangeWatcher.State}. + * @return the current state. + */ + public synchronized State getState() { + return state; + } + + /** + * Blocks until the current state becomes desiredState. Currently only used by tests, + * thus package-private. + * @param desiredState the desired state. + * @throws InterruptedException if the current thread gets interrupted. + */ + synchronized void waitForState(State desiredState) throws InterruptedException { + while (this.state != desiredState) { + this.wait(); + } + } + + /** + * Sets the state to newState. + * @param newState the new state. + */ + private synchronized void setState(State newState) { + state = newState; + this.notifyAll(); + } + + /** + * Atomically sets the state to update if and only if the state is currently + * expected. + * @param expected the expected state. + * @param update the new state. + * @return true if the update succeeds, or false if the current state does not equal + * expected. + */ + private synchronized boolean compareAndSetState(State expected, State update) { + if (state == expected) { + setState(update); + return true; + } else { + return false; + } + } + + /** + * Atomically sets the state to update if and only if the state is currently one of + * expectedStates. + * @param expectedStates the expected states. + * @param update the new state. + * @return true if the update succeeds, or false if the current state does not equal any of the + * expectedStates. + */ + private synchronized boolean compareAndSetState(State[] expectedStates, State update) { + for (State expected : expectedStates) { + if (state == expected) { + setState(update); + return true; + } + } + return false; + } + + /** + * Tells the background thread to start. Does not wait for it to be running. Calling this method + * more than once has no effect. + */ + public void start() { + if (!compareAndSetState(State.NEW, State.STARTING)) { + // If previous state was not NEW, start() has already been called. + return; + } + this.watcherThread.start(); + } + + /** + * Tells the background thread to stop. Does not wait for it to exit. + */ + public void stop() { + if (compareAndSetState(new State[] { State.RUNNING, State.STARTING }, State.STOPPING)) { + watcherThread.interrupt(); + } + } + + /** + * Inner class that implements the watcher thread logic. + */ + private class WatcherThread extends ZooKeeperThread { + + private static final String THREAD_NAME = "FileChangeWatcher"; + + final WatchService watchService; + final Consumer> callback; + + WatcherThread(WatchService watchService, Consumer> callback) { + super(THREAD_NAME); + this.watchService = watchService; + this.callback = callback; + } + + @Override + public void run() { + try { + LOG.info("{} thread started", getName()); + if ( + !compareAndSetState(FileChangeWatcher.State.STARTING, FileChangeWatcher.State.RUNNING) + ) { + // stop() called shortly after start(), before + // this thread started running. + FileChangeWatcher.State state = FileChangeWatcher.this.getState(); + if (state != FileChangeWatcher.State.STOPPING) { + throw new IllegalStateException("Unexpected state: " + state); + } + return; + } + runLoop(); + } catch (Exception e) { + LOG.warn("Error in runLoop()", e); + throw e; + } finally { + try { + watchService.close(); + } catch (IOException e) { + LOG.warn("Error closing watch service", e); + } + LOG.info("{} thread finished", getName()); + FileChangeWatcher.this.setState(FileChangeWatcher.State.STOPPED); + } + } + + private void runLoop() { + while (FileChangeWatcher.this.getState() == FileChangeWatcher.State.RUNNING) { + WatchKey key; + try { + key = watchService.take(); + } catch (InterruptedException | ClosedWatchServiceException e) { + LOG.debug("{} was interrupted and is shutting down...", getName()); + break; + } + for (WatchEvent event : key.pollEvents()) { + LOG.debug("Got file changed event: {} with context: {}", event.kind(), event.context()); + try { + callback.accept(event); + } catch (Throwable e) { + LOG.error("Error from callback", e); + } + } + boolean isKeyValid = key.reset(); + if (!isKeyValid) { + // This is likely a problem, it means that file reloading is broken, probably because the + // directory we are watching was deleted or otherwise became inaccessible (unmounted, + // permissions + // changed, ???). + // For now, we log an error and exit the watcher thread. + LOG.error("Watch key no longer valid, maybe the directory is inaccessible?"); + break; + } + } + } + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/BCFKSFileLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/BCFKSFileLoader.java new file mode 100644 index 000000000000..cefa4135c90e --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/BCFKSFileLoader.java @@ -0,0 +1,42 @@ +/* + * 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.io.crypto.tls; + +/** + * Implementation of {@link FileKeyStoreLoader} that loads from BCKFS files. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +final class BCFKSFileLoader extends StandardTypeFileKeyStoreLoader { + private BCFKSFileLoader(String keyStorePath, String trustStorePath, char[] keyStorePassword, + char[] trustStorePassword) { + super(keyStorePath, trustStorePath, keyStorePassword, trustStorePassword, + SupportedStandardKeyFormat.BCFKS); + } + + static class Builder extends FileKeyStoreLoader.Builder { + @Override + BCFKSFileLoader build() { + return new BCFKSFileLoader(keyStorePath, trustStorePath, keyStorePassword, + trustStorePassword); + } + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/FileKeyStoreLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/FileKeyStoreLoader.java new file mode 100644 index 000000000000..3a1740b4faf0 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/FileKeyStoreLoader.java @@ -0,0 +1,80 @@ +/* + * 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.io.crypto.tls; + +import java.util.Objects; + +/** + * Base class for instances of {@link KeyStoreLoader} which load the key/trust stores from files on + * a filesystem. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +abstract class FileKeyStoreLoader implements KeyStoreLoader { + final String keyStorePath; + final String trustStorePath; + final char[] keyStorePassword; + final char[] trustStorePassword; + + FileKeyStoreLoader(String keyStorePath, String trustStorePath, char[] keyStorePassword, + char[] trustStorePassword) { + this.keyStorePath = keyStorePath; + this.trustStorePath = trustStorePath; + this.keyStorePassword = keyStorePassword; + this.trustStorePassword = trustStorePassword; + } + + /** + * Base class for builder pattern used by subclasses. + * @param the subtype of FileKeyStoreLoader created by the Builder. + */ + static abstract class Builder { + String keyStorePath; + String trustStorePath; + char[] keyStorePassword; + char[] trustStorePassword; + + Builder() { + } + + Builder setKeyStorePath(String keyStorePath) { + this.keyStorePath = Objects.requireNonNull(keyStorePath); + return this; + } + + Builder setTrustStorePath(String trustStorePath) { + this.trustStorePath = Objects.requireNonNull(trustStorePath); + return this; + } + + Builder setKeyStorePassword(char[] keyStorePassword) { + this.keyStorePassword = Objects.requireNonNull(keyStorePassword); + return this; + } + + Builder setTrustStorePassword(char[] trustStorePassword) { + this.trustStorePassword = Objects.requireNonNull(trustStorePassword); + return this; + } + + abstract T build(); + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/FileKeyStoreLoaderBuilderProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/FileKeyStoreLoaderBuilderProvider.java new file mode 100644 index 000000000000..432c8a06d114 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/FileKeyStoreLoaderBuilderProvider.java @@ -0,0 +1,54 @@ +/* + * 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.io.crypto.tls; + +import java.util.Objects; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +final class FileKeyStoreLoaderBuilderProvider { + /** + * Returns a {@link FileKeyStoreLoader.Builder} that can build a loader which loads keys and certs + * from files of the given {@link KeyStoreFileType}. + * @param type the file type to load keys/certs from. + * @return a new Builder. + */ + static FileKeyStoreLoader.Builder + getBuilderForKeyStoreFileType(KeyStoreFileType type) { + switch (Objects.requireNonNull(type)) { + case JKS: + return new JKSFileLoader.Builder(); + case PEM: + return new PEMFileLoader.Builder(); + case PKCS12: + return new PKCS12FileLoader.Builder(); + case BCFKS: + return new BCFKSFileLoader.Builder(); + default: + throw new AssertionError("Unexpected StoreFileType: " + type.name()); + } + } + + private FileKeyStoreLoaderBuilderProvider() { + // disabled + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/HBaseHostnameVerifier.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/HBaseHostnameVerifier.java new file mode 100644 index 000000000000..a703f5ff630e --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/HBaseHostnameVerifier.java @@ -0,0 +1,286 @@ +/* + * 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.io.crypto.tls; + +import java.net.InetAddress; +import java.security.cert.Certificate; +import java.security.cert.CertificateParsingException; +import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.Optional; +import javax.naming.InvalidNameException; +import javax.naming.NamingException; +import javax.naming.directory.Attribute; +import javax.naming.directory.Attributes; +import javax.naming.ldap.LdapName; +import javax.naming.ldap.Rdn; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLException; +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLSession; +import javax.security.auth.x500.X500Principal; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.net.InetAddresses; + +/** + * When enabled in {@link X509Util}, handles verifying that the hostname of a peer matches the + * certificate it presents. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +final class HBaseHostnameVerifier implements HostnameVerifier { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseHostnameVerifier.class); + + /** + * Note: copied from Apache httpclient with some minor modifications. We want host verification, + * but depending on the httpclient jar caused unexplained performance regressions (even when the + * code was not used). + */ + private static final class SubjectName { + + static final int DNS = 2; + static final int IP = 7; + + private final String value; + private final int type; + + SubjectName(final String value, final int type) { + if (type != DNS && type != IP) { + throw new IllegalArgumentException("Invalid type: " + type); + } + this.value = Objects.requireNonNull(value); + this.type = type; + } + + public int getType() { + return type; + } + + public String getValue() { + return value; + } + + @Override + public String toString() { + return value; + } + + } + + @Override + public boolean verify(final String host, final SSLSession session) { + try { + final Certificate[] certs = session.getPeerCertificates(); + final X509Certificate x509 = (X509Certificate) certs[0]; + verify(host, x509); + return true; + } catch (final SSLException ex) { + LOG.debug("Unexpected exception", ex); + return false; + } + } + + void verify(final String host, final X509Certificate cert) throws SSLException { + final List subjectAlts = getSubjectAltNames(cert); + if (subjectAlts != null && !subjectAlts.isEmpty()) { + Optional inetAddress = parseIpAddress(host); + if (inetAddress.isPresent()) { + matchIPAddress(host, inetAddress.get(), subjectAlts); + } else { + matchDNSName(host, subjectAlts); + } + } else { + // CN matching has been deprecated by rfc2818 and can be used + // as fallback only when no subjectAlts are available + final X500Principal subjectPrincipal = cert.getSubjectX500Principal(); + final String cn = extractCN(subjectPrincipal.getName(X500Principal.RFC2253)); + if (cn == null) { + throw new SSLException("Certificate subject for <" + host + "> doesn't contain " + + "a common name and does not have alternative names"); + } + matchCN(host, cn); + } + } + + private static void matchIPAddress(final String host, final InetAddress inetAddress, + final List subjectAlts) throws SSLException { + for (final SubjectName subjectAlt : subjectAlts) { + if (subjectAlt.getType() == SubjectName.IP) { + Optional parsed = parseIpAddress(subjectAlt.getValue()); + if (parsed.filter(altAddr -> altAddr.equals(inetAddress)).isPresent()) { + return; + } + } + } + throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " + + "of the subject alternative names: " + subjectAlts); + } + + private static void matchDNSName(final String host, final List subjectAlts) + throws SSLException { + final String normalizedHost = host.toLowerCase(Locale.ROOT); + for (final SubjectName subjectAlt : subjectAlts) { + if (subjectAlt.getType() == SubjectName.DNS) { + final String normalizedSubjectAlt = subjectAlt.getValue().toLowerCase(Locale.ROOT); + if (matchIdentityStrict(normalizedHost, normalizedSubjectAlt)) { + return; + } + } + } + throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " + + "of the subject alternative names: " + subjectAlts); + } + + private static void matchCN(final String host, final String cn) throws SSLException { + final String normalizedHost = host.toLowerCase(Locale.ROOT); + final String normalizedCn = cn.toLowerCase(Locale.ROOT); + if (!matchIdentityStrict(normalizedHost, normalizedCn)) { + throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match " + + "common name of the certificate subject: " + cn); + } + } + + private static boolean matchIdentity(final String host, final String identity, + final boolean strict) { + // RFC 2818, 3.1. Server Identity + // "...Names may contain the wildcard + // character * which is considered to match any single domain name + // component or component fragment..." + // Based on this statement presuming only singular wildcard is legal + final int asteriskIdx = identity.indexOf('*'); + if (asteriskIdx != -1) { + final String prefix = identity.substring(0, asteriskIdx); + final String suffix = identity.substring(asteriskIdx + 1); + if (!prefix.isEmpty() && !host.startsWith(prefix)) { + return false; + } + if (!suffix.isEmpty() && !host.endsWith(suffix)) { + return false; + } + // Additional sanity checks on content selected by wildcard can be done here + if (strict) { + final String remainder = host.substring(prefix.length(), host.length() - suffix.length()); + return !remainder.contains("."); + } + return true; + } + return host.equalsIgnoreCase(identity); + } + + private static boolean matchIdentityStrict(final String host, final String identity) { + return matchIdentity(host, identity, true); + } + + private static String extractCN(final String subjectPrincipal) throws SSLException { + if (subjectPrincipal == null) { + return null; + } + try { + final LdapName subjectDN = new LdapName(subjectPrincipal); + final List rdns = subjectDN.getRdns(); + for (int i = rdns.size() - 1; i >= 0; i--) { + final Rdn rds = rdns.get(i); + final Attributes attributes = rds.toAttributes(); + final Attribute cn = attributes.get("cn"); + if (cn != null) { + try { + final Object value = cn.get(); + if (value != null) { + return value.toString(); + } + } catch (final NoSuchElementException ignore) { + // ignore exception + } catch (final NamingException ignore) { + // ignore exception + } + } + } + return null; + } catch (final InvalidNameException e) { + throw new SSLException(subjectPrincipal + " is not a valid X500 distinguished name"); + } + } + + private static Optional parseIpAddress(String host) { + host = host.trim(); + // Uri strings only work for ipv6 and are wrapped with brackets + // Unfortunately InetAddresses can't handle a mixed input, so we + // check here and choose which parse method to use. + if (host.startsWith("[") && host.endsWith("]")) { + return parseIpAddressUriString(host); + } else { + return parseIpAddressString(host); + } + } + + private static Optional parseIpAddressUriString(String host) { + if (InetAddresses.isUriInetAddress(host)) { + return Optional.of(InetAddresses.forUriString(host)); + } + return Optional.empty(); + } + + private static Optional parseIpAddressString(String host) { + if (InetAddresses.isInetAddress(host)) { + return Optional.of(InetAddresses.forString(host)); + } + return Optional.empty(); + } + + @SuppressWarnings("MixedMutabilityReturnType") + private static List getSubjectAltNames(final X509Certificate cert) { + try { + final Collection> entries = cert.getSubjectAlternativeNames(); + if (entries == null) { + return Collections.emptyList(); + } + final List result = new ArrayList(); + for (List entry : entries) { + final Integer type = entry.size() >= 2 ? (Integer) entry.get(0) : null; + if (type != null) { + if (type == SubjectName.DNS || type == SubjectName.IP) { + final Object o = entry.get(1); + if (o instanceof String) { + result.add(new SubjectName((String) o, type)); + } else { + LOG.debug("non-string Subject Alt Name type detected, not currently supported: {}", + o); + } + } + } + } + return result; + } catch (final CertificateParsingException ignore) { + return Collections.emptyList(); + } + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/HBaseTrustManager.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/HBaseTrustManager.java new file mode 100644 index 000000000000..ca4756a6131c --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/HBaseTrustManager.java @@ -0,0 +1,181 @@ +/* + * 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.io.crypto.tls; + +import java.net.InetAddress; +import java.net.Socket; +import java.net.UnknownHostException; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLException; +import javax.net.ssl.X509ExtendedTrustManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A custom TrustManager that supports hostname verification We attempt to perform verification + * using just the IP address first and if that fails will attempt to perform a reverse DNS lookup + * and verify using the hostname. This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +public class HBaseTrustManager extends X509ExtendedTrustManager { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseTrustManager.class); + + private final X509ExtendedTrustManager x509ExtendedTrustManager; + private final boolean hostnameVerificationEnabled; + private final boolean allowReverseDnsLookup; + + private final HBaseHostnameVerifier hostnameVerifier; + + /** + * Instantiate a new HBaseTrustManager. + * @param x509ExtendedTrustManager The trustmanager to use for + * checkClientTrusted/checkServerTrusted logic + * @param hostnameVerificationEnabled If true, this TrustManager should verify hostnames of peers + * when checking trust. + * @param allowReverseDnsLookup If true, we will fall back on reverse dns if resolving of + * host fails + */ + HBaseTrustManager(X509ExtendedTrustManager x509ExtendedTrustManager, + boolean hostnameVerificationEnabled, boolean allowReverseDnsLookup) { + this.x509ExtendedTrustManager = x509ExtendedTrustManager; + this.hostnameVerificationEnabled = hostnameVerificationEnabled; + this.allowReverseDnsLookup = allowReverseDnsLookup; + this.hostnameVerifier = new HBaseHostnameVerifier(); + } + + @Override + public X509Certificate[] getAcceptedIssuers() { + return x509ExtendedTrustManager.getAcceptedIssuers(); + } + + @Override + public void checkClientTrusted(X509Certificate[] chain, String authType, Socket socket) + throws CertificateException { + x509ExtendedTrustManager.checkClientTrusted(chain, authType, socket); + if (hostnameVerificationEnabled) { + performHostVerification(socket.getInetAddress(), chain[0]); + } + } + + @Override + public void checkServerTrusted(X509Certificate[] chain, String authType, Socket socket) + throws CertificateException { + x509ExtendedTrustManager.checkServerTrusted(chain, authType, socket); + if (hostnameVerificationEnabled) { + performHostVerification(socket.getInetAddress(), chain[0]); + } + } + + @Override + public void checkClientTrusted(X509Certificate[] chain, String authType, SSLEngine engine) + throws CertificateException { + x509ExtendedTrustManager.checkClientTrusted(chain, authType, engine); + if (hostnameVerificationEnabled) { + try { + performHostVerification(InetAddress.getByName(engine.getPeerHost()), chain[0]); + } catch (UnknownHostException e) { + throw new CertificateException("Failed to verify host", e); + } + } + } + + @Override + public void checkServerTrusted(X509Certificate[] chain, String authType, SSLEngine engine) + throws CertificateException { + x509ExtendedTrustManager.checkServerTrusted(chain, authType, engine); + if (hostnameVerificationEnabled) { + try { + performHostVerification(InetAddress.getByName(engine.getPeerHost()), chain[0]); + } catch (UnknownHostException e) { + throw new CertificateException("Failed to verify host", e); + } + } + } + + @Override + public void checkClientTrusted(X509Certificate[] chain, String authType) + throws CertificateException { + x509ExtendedTrustManager.checkClientTrusted(chain, authType); + } + + @Override + public void checkServerTrusted(X509Certificate[] chain, String authType) + throws CertificateException { + x509ExtendedTrustManager.checkServerTrusted(chain, authType); + } + + /** + * Compares peer's hostname with the one stored in the provided client certificate. Performs + * verification with the help of provided HostnameVerifier. + * @param inetAddress Peer's inet address. + * @param certificate Peer's certificate + * @throws CertificateException Thrown if the provided certificate doesn't match the peer + * hostname. + */ + private void performHostVerification(InetAddress inetAddress, X509Certificate certificate) + throws CertificateException { + String hostAddress = ""; + String hostName = ""; + try { + hostAddress = inetAddress.getHostAddress(); + hostnameVerifier.verify(hostAddress, certificate); + } catch (SSLException addressVerificationException) { + // If we fail with hostAddress, we should try the hostname. + // The inetAddress may have been created with a hostname, in which case getHostName() will + // return quickly below. If not, a reverse lookup will happen, which can be expensive. + // We provide the option to skip the reverse lookup if preferring to fail fast. + + // Handle logging here to aid debugging. The easiest way to check for an existing + // hostname is through toString, see javadoc. + String inetAddressString = inetAddress.toString(); + if (!inetAddressString.startsWith("/")) { + LOG.debug( + "Failed to verify host address: {}, but inetAddress {} has a hostname, trying that", + hostAddress, inetAddressString, addressVerificationException); + } else if (allowReverseDnsLookup) { + LOG.debug( + "Failed to verify host address: {}, attempting to verify host name with reverse dns", + hostAddress, addressVerificationException); + } else { + LOG.debug("Failed to verify host address: {}, but reverse dns lookup is disabled", + hostAddress, addressVerificationException); + throw new CertificateException( + "Failed to verify host address, and reverse lookup is disabled", + addressVerificationException); + } + + try { + hostName = inetAddress.getHostName(); + hostnameVerifier.verify(hostName, certificate); + } catch (SSLException hostnameVerificationException) { + LOG.error("Failed to verify host address: {}", hostAddress, addressVerificationException); + LOG.error("Failed to verify hostname: {}", hostName, hostnameVerificationException); + throw new CertificateException("Failed to verify both host address and host name", + hostnameVerificationException); + } + } + } + +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/JKSFileLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/JKSFileLoader.java new file mode 100644 index 000000000000..36e9643cbfe6 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/JKSFileLoader.java @@ -0,0 +1,41 @@ +/* + * 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.io.crypto.tls; + +/** + * Implementation of {@link FileKeyStoreLoader} that loads from JKS files. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +final class JKSFileLoader extends StandardTypeFileKeyStoreLoader { + private JKSFileLoader(String keyStorePath, String trustStorePath, char[] keyStorePassword, + char[] trustStorePassword) { + super(keyStorePath, trustStorePath, keyStorePassword, trustStorePassword, + SupportedStandardKeyFormat.JKS); + } + + static class Builder extends FileKeyStoreLoader.Builder { + @Override + JKSFileLoader build() { + return new JKSFileLoader(keyStorePath, trustStorePath, keyStorePassword, trustStorePassword); + } + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/KeyStoreFileType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/KeyStoreFileType.java new file mode 100644 index 000000000000..851b68124236 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/KeyStoreFileType.java @@ -0,0 +1,122 @@ +/* + * 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.io.crypto.tls; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * This enum represents the file type of a KeyStore or TrustStore. Currently, JKS (Java keystore), + * PEM, PKCS12, and BCFKS types are supported. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +public enum KeyStoreFileType { + JKS(".jks"), + PEM(".pem"), + PKCS12(".p12"), + BCFKS(".bcfks"); + + private final String defaultFileExtension; + + KeyStoreFileType(String defaultFileExtension) { + this.defaultFileExtension = defaultFileExtension; + } + + /** + * The property string that specifies that a key store or trust store should use this store file + * type. + */ + public String getPropertyValue() { + return this.name(); + } + + /** + * The file extension that is associated with this file type. + */ + public String getDefaultFileExtension() { + return defaultFileExtension; + } + + /** + * Converts a property value to a StoreFileType enum. If the property value is null + * or an empty string, returns null. + * @param propertyValue the property value. + * @return the KeyStoreFileType, or null if propertyValue is + * null or empty. + * @throws IllegalArgumentException if propertyValue is not one of "JKS", "PEM", + * "BCFKS", "PKCS12", or empty/null. + */ + public static KeyStoreFileType fromPropertyValue(String propertyValue) { + if (propertyValue == null || propertyValue.length() == 0) { + return null; + } + return KeyStoreFileType.valueOf(propertyValue.toUpperCase()); + } + + /** + * Detects the type of KeyStore / TrustStore file from the file extension. If the file name ends + * with ".jks", returns StoreFileType.JKS. If the file name ends with ".pem", returns + * StoreFileType.PEM. If the file name ends with ".p12", returns + * StoreFileType.PKCS12. If the file name ends with ".bckfs", returns + * StoreFileType.BCKFS. Otherwise, throws an IllegalArgumentException. + * @param filename the filename of the key store or trust store file. + * @return a KeyStoreFileType. + * @throws IllegalArgumentException if the filename does not end with ".jks", ".pem", "p12" or + * "bcfks". + */ + public static KeyStoreFileType fromFilename(String filename) { + int i = filename.lastIndexOf('.'); + if (i >= 0) { + String extension = filename.substring(i); + for (KeyStoreFileType storeFileType : KeyStoreFileType.values()) { + if (storeFileType.getDefaultFileExtension().equals(extension)) { + return storeFileType; + } + } + } + throw new IllegalArgumentException( + "Unable to auto-detect store file type from file name: " + filename); + } + + /** + * If propertyValue is not null or empty, returns the result of + * KeyStoreFileType.fromPropertyValue(propertyValue). Else, returns the result of + * KeyStoreFileType.fromFileName(filename). + * @param propertyValue property value describing the KeyStoreFileType, or null/empty to + * auto-detect the type from the file name. + * @param filename file name of the key store file. The file extension is used to auto-detect + * the KeyStoreFileType when propertyValue is null or empty. + * @return a KeyStoreFileType. + * @throws IllegalArgumentException if propertyValue is not one of "JKS", "PEM", + * "PKCS12", "BCFKS", or empty/null. + * @throws IllegalArgumentException if propertyValueis empty or null and the type + * could not be determined from the file name. + */ + public static KeyStoreFileType fromPropertyValueOrFileName(String propertyValue, + String filename) { + KeyStoreFileType result = KeyStoreFileType.fromPropertyValue(propertyValue); + if (result == null) { + result = KeyStoreFileType.fromFilename(filename); + } + return result; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/KeyStoreLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/KeyStoreLoader.java new file mode 100644 index 000000000000..928b3b9d0469 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/KeyStoreLoader.java @@ -0,0 +1,53 @@ +/* + * 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.io.crypto.tls; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; + +/** + * An interface for an object that can load key stores or trust stores. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +interface KeyStoreLoader { + /** + * Loads a KeyStore which contains at least one private key and the associated X509 cert chain. + * @return a new KeyStore + * @throws IOException if loading the key store fails due to an IO error, such as + * "file not found". + * @throws GeneralSecurityException if loading the key store fails due to a security error, such + * as "unsupported crypto algorithm". + */ + KeyStore loadKeyStore() throws IOException, GeneralSecurityException; + + /** + * Loads a KeyStore which contains at least one X509 cert chain for a trusted Certificate + * Authority (CA). + * @return a new KeyStore + * @throws IOException if loading the trust store fails due to an IO error, such as + * "file not found". + * @throws GeneralSecurityException if loading the trust store fails due to a security error, such + * as "unsupported crypto algorithm". + */ + KeyStore loadTrustStore() throws IOException, GeneralSecurityException; +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PEMFileLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PEMFileLoader.java new file mode 100644 index 000000000000..06d264b67da2 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PEMFileLoader.java @@ -0,0 +1,56 @@ +/* + * 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.io.crypto.tls; + +import java.io.File; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; + +/** + * Implementation of {@link FileKeyStoreLoader} that loads from PEM files. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +final class PEMFileLoader extends FileKeyStoreLoader { + private PEMFileLoader(String keyStorePath, String trustStorePath, char[] keyStorePassword, + char[] trustStorePassword) { + super(keyStorePath, trustStorePath, keyStorePassword, trustStorePassword); + } + + @Override + public KeyStore loadKeyStore() throws IOException, GeneralSecurityException { + File file = new File(keyStorePath); + return PemReader.loadKeyStore(file, file, keyStorePassword); + } + + @Override + public KeyStore loadTrustStore() throws IOException, GeneralSecurityException { + return PemReader.loadTrustStore(new File(trustStorePath)); + } + + static class Builder extends FileKeyStoreLoader.Builder { + @Override + PEMFileLoader build() { + return new PEMFileLoader(keyStorePath, trustStorePath, keyStorePassword, trustStorePassword); + } + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PKCS12FileLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PKCS12FileLoader.java new file mode 100644 index 000000000000..ab5a532787e0 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PKCS12FileLoader.java @@ -0,0 +1,42 @@ +/* + * 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.io.crypto.tls; + +/** + * Implementation of {@link FileKeyStoreLoader} that loads from PKCS12 files. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +final class PKCS12FileLoader extends StandardTypeFileKeyStoreLoader { + private PKCS12FileLoader(String keyStorePath, String trustStorePath, char[] keyStorePassword, + char[] trustStorePassword) { + super(keyStorePath, trustStorePath, keyStorePassword, trustStorePassword, + SupportedStandardKeyFormat.PKCS12); + } + + static class Builder extends FileKeyStoreLoader.Builder { + @Override + PKCS12FileLoader build() { + return new PKCS12FileLoader(keyStorePath, trustStorePath, keyStorePassword, + trustStorePassword); + } + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PemReader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PemReader.java new file mode 100644 index 000000000000..b4f7aa5565a2 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/PemReader.java @@ -0,0 +1,218 @@ +/* + * 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.io.crypto.tls; + +import static java.nio.charset.StandardCharsets.US_ASCII; +import static java.util.Base64.getMimeDecoder; +import static java.util.regex.Pattern.CASE_INSENSITIVE; +import static javax.crypto.Cipher.DECRYPT_MODE; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.security.GeneralSecurityException; +import java.security.KeyFactory; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.PrivateKey; +import java.security.PublicKey; +import java.security.cert.Certificate; +import java.security.cert.CertificateException; +import java.security.cert.CertificateFactory; +import java.security.cert.X509Certificate; +import java.security.spec.InvalidKeySpecException; +import java.security.spec.PKCS8EncodedKeySpec; +import java.security.spec.X509EncodedKeySpec; +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.crypto.Cipher; +import javax.crypto.EncryptedPrivateKeyInfo; +import javax.crypto.SecretKey; +import javax.crypto.SecretKeyFactory; +import javax.crypto.spec.PBEKeySpec; +import javax.security.auth.x500.X500Principal; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +final class PemReader { + private static final Pattern CERT_PATTERN = + Pattern.compile("-+BEGIN\\s+.*CERTIFICATE[^-]*-+(?:\\s|\\r|\\n)+" + // Header + "([a-z0-9+/=\\r\\n]+)" + // Base64 text + "-+END\\s+.*CERTIFICATE[^-]*-+", // Footer + CASE_INSENSITIVE); + + private static final Pattern PRIVATE_KEY_PATTERN = + Pattern.compile("-+BEGIN\\s+.*PRIVATE\\s+KEY[^-]*-+(?:\\s|\\r|\\n)+" + // Header + "([a-z0-9+/=\\r\\n]+)" + // Base64 text + "-+END\\s+.*PRIVATE\\s+KEY[^-]*-+", // Footer + CASE_INSENSITIVE); + + private static final Pattern PUBLIC_KEY_PATTERN = + Pattern.compile("-+BEGIN\\s+.*PUBLIC\\s+KEY[^-]*-+(?:\\s|\\r|\\n)+" + // Header + "([a-z0-9+/=\\r\\n]+)" + // Base64 text + "-+END\\s+.*PUBLIC\\s+KEY[^-]*-+", // Footer + CASE_INSENSITIVE); + + private PemReader() { + } + + public static KeyStore loadTrustStore(File certificateChainFile) + throws IOException, GeneralSecurityException { + KeyStore keyStore = KeyStore.getInstance("JKS"); + keyStore.load(null, null); + + List certificateChain = readCertificateChain(certificateChainFile); + for (X509Certificate certificate : certificateChain) { + X500Principal principal = certificate.getSubjectX500Principal(); + keyStore.setCertificateEntry(principal.getName("RFC2253"), certificate); + } + return keyStore; + } + + public static KeyStore loadKeyStore(File certificateChainFile, File privateKeyFile, + char[] keyPassword) throws IOException, GeneralSecurityException { + PrivateKey key = loadPrivateKey(privateKeyFile, keyPassword); + + List certificateChain = readCertificateChain(certificateChainFile); + if (certificateChain.isEmpty()) { + throw new CertificateException( + "Certificate file does not contain any certificates: " + certificateChainFile); + } + + KeyStore keyStore = KeyStore.getInstance("JKS"); + keyStore.load(null, null); + keyStore.setKeyEntry("key", key, keyPassword, certificateChain.toArray(new Certificate[0])); + return keyStore; + } + + public static List readCertificateChain(File certificateChainFile) + throws IOException, GeneralSecurityException { + String contents = new String(Files.readAllBytes(certificateChainFile.toPath()), US_ASCII); + return readCertificateChain(contents); + } + + public static List readCertificateChain(String certificateChain) + throws CertificateException { + Matcher matcher = CERT_PATTERN.matcher(certificateChain); + CertificateFactory certificateFactory = CertificateFactory.getInstance("X.509"); + List certificates = new ArrayList<>(); + + int start = 0; + while (matcher.find(start)) { + byte[] buffer = base64Decode(matcher.group(1)); + certificates.add( + (X509Certificate) certificateFactory.generateCertificate(new ByteArrayInputStream(buffer))); + start = matcher.end(); + } + + return certificates; + } + + public static PrivateKey loadPrivateKey(File privateKeyFile, char[] keyPassword) + throws IOException, GeneralSecurityException { + String privateKey = new String(Files.readAllBytes(privateKeyFile.toPath()), US_ASCII); + return loadPrivateKey(privateKey, keyPassword); + } + + public static PrivateKey loadPrivateKey(String privateKey, char[] keyPassword) + throws IOException, GeneralSecurityException { + Matcher matcher = PRIVATE_KEY_PATTERN.matcher(privateKey); + if (!matcher.find()) { + throw new KeyStoreException("did not find a private key"); + } + byte[] encodedKey = base64Decode(matcher.group(1)); + + PKCS8EncodedKeySpec encodedKeySpec; + if (keyPassword != null && keyPassword.length > 0) { + EncryptedPrivateKeyInfo encryptedPrivateKeyInfo = new EncryptedPrivateKeyInfo(encodedKey); + SecretKeyFactory keyFactory = + SecretKeyFactory.getInstance(encryptedPrivateKeyInfo.getAlgName()); + SecretKey secretKey = keyFactory.generateSecret(new PBEKeySpec(keyPassword)); + + Cipher cipher = Cipher.getInstance(encryptedPrivateKeyInfo.getAlgName()); + cipher.init(DECRYPT_MODE, secretKey, encryptedPrivateKeyInfo.getAlgParameters()); + + encodedKeySpec = encryptedPrivateKeyInfo.getKeySpec(cipher); + } else { + encodedKeySpec = new PKCS8EncodedKeySpec(encodedKey); + } + + // this code requires a key in PKCS8 format which is not the default openssl format + // to convert to the PKCS8 format you use : openssl pkcs8 -topk8 ... + try { + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + return keyFactory.generatePrivate(encodedKeySpec); + } catch (InvalidKeySpecException ignore) { + // ignore + } + + try { + KeyFactory keyFactory = KeyFactory.getInstance("EC"); + return keyFactory.generatePrivate(encodedKeySpec); + } catch (InvalidKeySpecException ignore) { + // ignore + } + + KeyFactory keyFactory = KeyFactory.getInstance("DSA"); + return keyFactory.generatePrivate(encodedKeySpec); + } + + public static PublicKey loadPublicKey(File publicKeyFile) + throws IOException, GeneralSecurityException { + String publicKey = new String(Files.readAllBytes(publicKeyFile.toPath()), US_ASCII); + return loadPublicKey(publicKey); + } + + public static PublicKey loadPublicKey(String publicKey) throws GeneralSecurityException { + Matcher matcher = PUBLIC_KEY_PATTERN.matcher(publicKey); + if (!matcher.find()) { + throw new KeyStoreException("did not find a public key"); + } + String data = matcher.group(1); + byte[] encodedKey = base64Decode(data); + + X509EncodedKeySpec encodedKeySpec = new X509EncodedKeySpec(encodedKey); + try { + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + return keyFactory.generatePublic(encodedKeySpec); + } catch (InvalidKeySpecException ignore) { + // ignore + } + + try { + KeyFactory keyFactory = KeyFactory.getInstance("EC"); + return keyFactory.generatePublic(encodedKeySpec); + } catch (InvalidKeySpecException ignore) { + // ignore + } + + KeyFactory keyFactory = KeyFactory.getInstance("DSA"); + return keyFactory.generatePublic(encodedKeySpec); + } + + private static byte[] base64Decode(String base64) { + return getMimeDecoder().decode(base64.getBytes(US_ASCII)); + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/StandardTypeFileKeyStoreLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/StandardTypeFileKeyStoreLoader.java new file mode 100644 index 000000000000..67aebdd6c7bc --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/StandardTypeFileKeyStoreLoader.java @@ -0,0 +1,79 @@ +/* + * 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.io.crypto.tls; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.KeyStoreException; + +/** + * Base class for instances of {@link KeyStoreLoader} which load the key/trust stores from files on + * a filesystem using standard {@link KeyStore} types like JKS or PKCS12. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +abstract class StandardTypeFileKeyStoreLoader extends FileKeyStoreLoader { + private static final char[] EMPTY_CHAR_ARRAY = new char[0]; + + protected final SupportedStandardKeyFormat format; + + protected enum SupportedStandardKeyFormat { + JKS, + PKCS12, + BCFKS + } + + StandardTypeFileKeyStoreLoader(String keyStorePath, String trustStorePath, + char[] keyStorePassword, char[] trustStorePassword, SupportedStandardKeyFormat format) { + super(keyStorePath, trustStorePath, keyStorePassword, trustStorePassword); + this.format = format; + } + + @Override + public KeyStore loadKeyStore() throws IOException, GeneralSecurityException { + try (InputStream inputStream = Files.newInputStream(new File(keyStorePath).toPath())) { + KeyStore ks = keyStoreInstance(); + ks.load(inputStream, passwordStringToCharArray(keyStorePassword)); + return ks; + } + } + + @Override + public KeyStore loadTrustStore() throws IOException, GeneralSecurityException { + try (InputStream inputStream = Files.newInputStream(new File(trustStorePath).toPath())) { + KeyStore ts = keyStoreInstance(); + ts.load(inputStream, passwordStringToCharArray(trustStorePassword)); + return ts; + } + } + + private KeyStore keyStoreInstance() throws KeyStoreException { + return KeyStore.getInstance(format.name()); + } + + private static char[] passwordStringToCharArray(char[] password) { + return password == null ? EMPTY_CHAR_ARRAY : password; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java new file mode 100644 index 000000000000..7f46bb5f6d3b --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java @@ -0,0 +1,474 @@ +/* + * 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.io.crypto.tls; + +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.Security; +import java.security.cert.PKIXBuilderParameters; +import java.security.cert.X509CertSelector; +import java.util.Arrays; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; +import javax.net.ssl.CertPathTrustManagerParameters; +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.X509ExtendedTrustManager; +import javax.net.ssl.X509KeyManager; +import javax.net.ssl.X509TrustManager; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.exceptions.KeyManagerException; +import org.apache.hadoop.hbase.exceptions.SSLContextException; +import org.apache.hadoop.hbase.exceptions.TrustManagerException; +import org.apache.hadoop.hbase.exceptions.X509Exception; +import org.apache.hadoop.hbase.io.FileChangeWatcher; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.ObjectArrays; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslContextBuilder; + +/** + * Utility code for X509 handling Default cipher suites: Performance testing done by Facebook + * engineers shows that on Intel x86_64 machines, Java9 performs better with GCM and Java8 performs + * better with CBC, so these seem like reasonable defaults. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +public final class X509Util { + + private static final Logger LOG = LoggerFactory.getLogger(X509Util.class); + private static final char[] EMPTY_CHAR_ARRAY = new char[0]; + + // + // Common tls configs across both server and client + // + static final String CONFIG_PREFIX = "hbase.rpc.tls."; + public static final String TLS_CONFIG_PROTOCOL = CONFIG_PREFIX + "protocol"; + public static final String TLS_CONFIG_KEYSTORE_LOCATION = CONFIG_PREFIX + "keystore.location"; + public static final String TLS_CONFIG_KEYSTORE_TYPE = CONFIG_PREFIX + "keystore.type"; + public static final String TLS_CONFIG_KEYSTORE_PASSWORD = CONFIG_PREFIX + "keystore.password"; + public static final String TLS_CONFIG_TRUSTSTORE_LOCATION = CONFIG_PREFIX + "truststore.location"; + public static final String TLS_CONFIG_TRUSTSTORE_TYPE = CONFIG_PREFIX + "truststore.type"; + public static final String TLS_CONFIG_TRUSTSTORE_PASSWORD = CONFIG_PREFIX + "truststore.password"; + public static final String TLS_CONFIG_CLR = CONFIG_PREFIX + "clr"; + public static final String TLS_CONFIG_OCSP = CONFIG_PREFIX + "ocsp"; + public static final String TLS_CONFIG_REVERSE_DNS_LOOKUP_ENABLED = + CONFIG_PREFIX + "host-verification.reverse-dns.enabled"; + public static final String TLS_ENABLED_PROTOCOLS = CONFIG_PREFIX + "enabledProtocols"; + public static final String TLS_CIPHER_SUITES = CONFIG_PREFIX + "ciphersuites"; + public static final String TLS_CERT_RELOAD = CONFIG_PREFIX + "certReload"; + public static final String DEFAULT_PROTOCOL = "TLSv1.2"; + + // + // Server-side specific configs + // + public static final String HBASE_SERVER_NETTY_TLS_ENABLED = "hbase.server.netty.tls.enabled"; + public static final String HBASE_SERVER_NETTY_TLS_CLIENT_AUTH_MODE = + "hbase.server.netty.tls.client.auth.mode"; + public static final String HBASE_SERVER_NETTY_TLS_VERIFY_CLIENT_HOSTNAME = + "hbase.server.netty.tls.verify.client.hostname"; + public static final String HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT = + "hbase.server.netty.tls.supportplaintext"; + + // + // Client-side specific configs + // + public static final String HBASE_CLIENT_NETTY_TLS_ENABLED = "hbase.client.netty.tls.enabled"; + public static final String HBASE_CLIENT_NETTY_TLS_VERIFY_SERVER_HOSTNAME = + "hbase.client.netty.tls.verify.server.hostname"; + public static final String HBASE_CLIENT_NETTY_TLS_HANDSHAKETIMEOUT = + "hbase.client.netty.tls.handshaketimeout"; + public static final int DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS = 5000; + + private static String[] getGCMCiphers() { + return new String[] { "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256", + "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", + "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384" }; + } + + private static String[] getCBCCiphers() { + return new String[] { "TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256", + "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256", "TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA", + "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA", "TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA384", + "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384", "TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA", + "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA" }; + } + + // On Java 8, prefer CBC ciphers since AES-NI support is lacking and GCM is slower than CBC. + private static final String[] DEFAULT_CIPHERS_JAVA8 = + ObjectArrays.concat(getCBCCiphers(), getGCMCiphers(), String.class); + // On Java 9 and later, prefer GCM ciphers due to improved AES-NI support. + // Note that this performance assumption might not hold true for architectures other than x86_64. + private static final String[] DEFAULT_CIPHERS_JAVA9 = + ObjectArrays.concat(getGCMCiphers(), getCBCCiphers(), String.class); + + /** + * Enum specifying the client auth requirement of server-side TLS sockets created by this + * X509Util. + *

    + *
  • NONE - do not request a client certificate.
  • + *
  • WANT - request a client certificate, but allow anonymous clients to connect.
  • + *
  • NEED - require a client certificate, disconnect anonymous clients.
  • + *
+ * If the config property is not set, the default value is NEED. + */ + public enum ClientAuth { + NONE(org.apache.hbase.thirdparty.io.netty.handler.ssl.ClientAuth.NONE), + WANT(org.apache.hbase.thirdparty.io.netty.handler.ssl.ClientAuth.OPTIONAL), + NEED(org.apache.hbase.thirdparty.io.netty.handler.ssl.ClientAuth.REQUIRE); + + private final org.apache.hbase.thirdparty.io.netty.handler.ssl.ClientAuth nettyAuth; + + ClientAuth(org.apache.hbase.thirdparty.io.netty.handler.ssl.ClientAuth nettyAuth) { + this.nettyAuth = nettyAuth; + } + + /** + * Converts a property value to a ClientAuth enum. If the input string is empty or null, returns + * ClientAuth.NEED. + * @param prop the property string. + * @return the ClientAuth. + * @throws IllegalArgumentException if the property value is not "NONE", "WANT", "NEED", or + * empty/null. + */ + public static ClientAuth fromPropertyValue(String prop) { + if (prop == null || prop.length() == 0) { + return NEED; + } + return ClientAuth.valueOf(prop.toUpperCase()); + } + + public org.apache.hbase.thirdparty.io.netty.handler.ssl.ClientAuth toNettyClientAuth() { + return nettyAuth; + } + } + + private X509Util() { + // disabled + } + + static String[] getDefaultCipherSuites() { + return getDefaultCipherSuitesForJavaVersion(System.getProperty("java.specification.version")); + } + + static String[] getDefaultCipherSuitesForJavaVersion(String javaVersion) { + Objects.requireNonNull(javaVersion); + if (javaVersion.matches("\\d+")) { + // Must be Java 9 or later + LOG.debug("Using Java9+ optimized cipher suites for Java version {}", javaVersion); + return DEFAULT_CIPHERS_JAVA9; + } else if (javaVersion.startsWith("1.")) { + // Must be Java 1.8 or earlier + LOG.debug("Using Java8 optimized cipher suites for Java version {}", javaVersion); + return DEFAULT_CIPHERS_JAVA8; + } else { + LOG.debug("Could not parse java version {}, using Java8 optimized cipher suites", + javaVersion); + return DEFAULT_CIPHERS_JAVA8; + } + } + + public static SslContext createSslContextForClient(Configuration config) + throws X509Exception, IOException { + + SslContextBuilder sslContextBuilder = SslContextBuilder.forClient(); + + String keyStoreLocation = config.get(TLS_CONFIG_KEYSTORE_LOCATION, ""); + char[] keyStorePassword = config.getPassword(TLS_CONFIG_KEYSTORE_PASSWORD); + String keyStoreType = config.get(TLS_CONFIG_KEYSTORE_TYPE, ""); + + if (keyStoreLocation.isEmpty()) { + LOG.warn(TLS_CONFIG_KEYSTORE_LOCATION + " not specified"); + } else { + sslContextBuilder + .keyManager(createKeyManager(keyStoreLocation, keyStorePassword, keyStoreType)); + } + + String trustStoreLocation = config.get(TLS_CONFIG_TRUSTSTORE_LOCATION, ""); + char[] trustStorePassword = config.getPassword(TLS_CONFIG_TRUSTSTORE_PASSWORD); + String trustStoreType = config.get(TLS_CONFIG_TRUSTSTORE_TYPE, ""); + + boolean sslCrlEnabled = config.getBoolean(TLS_CONFIG_CLR, false); + boolean sslOcspEnabled = config.getBoolean(TLS_CONFIG_OCSP, false); + + boolean verifyServerHostname = + config.getBoolean(HBASE_CLIENT_NETTY_TLS_VERIFY_SERVER_HOSTNAME, true); + boolean allowReverseDnsLookup = config.getBoolean(TLS_CONFIG_REVERSE_DNS_LOOKUP_ENABLED, true); + + if (trustStoreLocation.isEmpty()) { + LOG.warn(TLS_CONFIG_TRUSTSTORE_LOCATION + " not specified"); + } else { + sslContextBuilder + .trustManager(createTrustManager(trustStoreLocation, trustStorePassword, trustStoreType, + sslCrlEnabled, sslOcspEnabled, verifyServerHostname, allowReverseDnsLookup)); + } + + sslContextBuilder.enableOcsp(sslOcspEnabled); + sslContextBuilder.protocols(getEnabledProtocols(config)); + sslContextBuilder.ciphers(Arrays.asList(getCipherSuites(config))); + + return sslContextBuilder.build(); + } + + public static SslContext createSslContextForServer(Configuration config) + throws X509Exception, IOException { + String keyStoreLocation = config.get(TLS_CONFIG_KEYSTORE_LOCATION, ""); + char[] keyStorePassword = config.getPassword(TLS_CONFIG_KEYSTORE_PASSWORD); + String keyStoreType = config.get(TLS_CONFIG_KEYSTORE_TYPE, ""); + + if (keyStoreLocation.isEmpty()) { + throw new SSLContextException( + "Keystore is required for SSL server: " + TLS_CONFIG_KEYSTORE_LOCATION); + } + + SslContextBuilder sslContextBuilder; + sslContextBuilder = SslContextBuilder + .forServer(createKeyManager(keyStoreLocation, keyStorePassword, keyStoreType)); + + String trustStoreLocation = config.get(TLS_CONFIG_TRUSTSTORE_LOCATION, ""); + char[] trustStorePassword = config.getPassword(TLS_CONFIG_TRUSTSTORE_PASSWORD); + String trustStoreType = config.get(TLS_CONFIG_TRUSTSTORE_TYPE, ""); + + boolean sslCrlEnabled = config.getBoolean(TLS_CONFIG_CLR, false); + boolean sslOcspEnabled = config.getBoolean(TLS_CONFIG_OCSP, false); + + ClientAuth clientAuth = + ClientAuth.fromPropertyValue(config.get(HBASE_SERVER_NETTY_TLS_CLIENT_AUTH_MODE)); + boolean verifyClientHostname = + config.getBoolean(HBASE_SERVER_NETTY_TLS_VERIFY_CLIENT_HOSTNAME, true); + boolean allowReverseDnsLookup = config.getBoolean(TLS_CONFIG_REVERSE_DNS_LOOKUP_ENABLED, true); + + if (trustStoreLocation.isEmpty()) { + LOG.warn(TLS_CONFIG_TRUSTSTORE_LOCATION + " not specified"); + } else { + sslContextBuilder + .trustManager(createTrustManager(trustStoreLocation, trustStorePassword, trustStoreType, + sslCrlEnabled, sslOcspEnabled, verifyClientHostname, allowReverseDnsLookup)); + } + + sslContextBuilder.enableOcsp(sslOcspEnabled); + sslContextBuilder.protocols(getEnabledProtocols(config)); + sslContextBuilder.ciphers(Arrays.asList(getCipherSuites(config))); + sslContextBuilder.clientAuth(clientAuth.toNettyClientAuth()); + + return sslContextBuilder.build(); + } + + /** + * Creates a key manager by loading the key store from the given file of the given type, + * optionally decrypting it using the given password. + * @param keyStoreLocation the location of the key store file. + * @param keyStorePassword optional password to decrypt the key store. If empty, assumes the key + * store is not encrypted. + * @param keyStoreType must be JKS, PEM, PKCS12, BCFKS or null. If null, attempts to + * autodetect the key store type from the file extension (e.g. .jks / + * .pem). + * @return the key manager. + * @throws KeyManagerException if something goes wrong. + */ + static X509KeyManager createKeyManager(String keyStoreLocation, char[] keyStorePassword, + String keyStoreType) throws KeyManagerException { + + if (keyStorePassword == null) { + keyStorePassword = EMPTY_CHAR_ARRAY; + } + + try { + KeyStoreFileType storeFileType = + KeyStoreFileType.fromPropertyValueOrFileName(keyStoreType, keyStoreLocation); + KeyStore ks = FileKeyStoreLoaderBuilderProvider.getBuilderForKeyStoreFileType(storeFileType) + .setKeyStorePath(keyStoreLocation).setKeyStorePassword(keyStorePassword).build() + .loadKeyStore(); + + KeyManagerFactory kmf = KeyManagerFactory.getInstance("PKIX"); + kmf.init(ks, keyStorePassword); + + for (KeyManager km : kmf.getKeyManagers()) { + if (km instanceof X509KeyManager) { + return (X509KeyManager) km; + } + } + throw new KeyManagerException("Couldn't find X509KeyManager"); + } catch (IOException | GeneralSecurityException | IllegalArgumentException e) { + throw new KeyManagerException(e); + } + } + + /** + * Creates a trust manager by loading the trust store from the given file of the given type, + * optionally decrypting it using the given password. + * @param trustStoreLocation the location of the trust store file. + * @param trustStorePassword optional password to decrypt the trust store (only applies to JKS + * trust stores). If empty, assumes the trust store is not encrypted. + * @param trustStoreType must be JKS, PEM, PKCS12, BCFKS or null. If null, attempts to + * autodetect the trust store type from the file extension (e.g. .jks + * / .pem). + * @param crlEnabled enable CRL (certificate revocation list) checks. + * @param ocspEnabled enable OCSP (online certificate status protocol) checks. + * @param verifyHostName if true, ssl peer hostname must match name in certificate + * @param allowReverseDnsLookup if true, allow falling back to reverse dns lookup in verifying + * hostname + * @return the trust manager. + * @throws TrustManagerException if something goes wrong. + */ + static X509TrustManager createTrustManager(String trustStoreLocation, char[] trustStorePassword, + String trustStoreType, boolean crlEnabled, boolean ocspEnabled, boolean verifyHostName, + boolean allowReverseDnsLookup) throws TrustManagerException { + + if (trustStorePassword == null) { + trustStorePassword = EMPTY_CHAR_ARRAY; + } + + try { + KeyStoreFileType storeFileType = + KeyStoreFileType.fromPropertyValueOrFileName(trustStoreType, trustStoreLocation); + KeyStore ts = FileKeyStoreLoaderBuilderProvider.getBuilderForKeyStoreFileType(storeFileType) + .setTrustStorePath(trustStoreLocation).setTrustStorePassword(trustStorePassword).build() + .loadTrustStore(); + + PKIXBuilderParameters pbParams = new PKIXBuilderParameters(ts, new X509CertSelector()); + if (crlEnabled || ocspEnabled) { + pbParams.setRevocationEnabled(true); + System.setProperty("com.sun.net.ssl.checkRevocation", "true"); + if (crlEnabled) { + System.setProperty("com.sun.security.enableCRLDP", "true"); + } + if (ocspEnabled) { + Security.setProperty("ocsp.enable", "true"); + } + } else { + pbParams.setRevocationEnabled(false); + } + + // Revocation checking is only supported with the PKIX algorithm + TrustManagerFactory tmf = TrustManagerFactory.getInstance("PKIX"); + tmf.init(new CertPathTrustManagerParameters(pbParams)); + + for (final TrustManager tm : tmf.getTrustManagers()) { + if (tm instanceof X509ExtendedTrustManager) { + return new HBaseTrustManager((X509ExtendedTrustManager) tm, verifyHostName, + allowReverseDnsLookup); + } + } + throw new TrustManagerException("Couldn't find X509TrustManager"); + } catch (IOException | GeneralSecurityException | IllegalArgumentException e) { + throw new TrustManagerException(e); + } + } + + private static String[] getEnabledProtocols(Configuration config) { + String enabledProtocolsInput = config.get(TLS_ENABLED_PROTOCOLS); + if (enabledProtocolsInput == null) { + return new String[] { config.get(TLS_CONFIG_PROTOCOL, DEFAULT_PROTOCOL) }; + } + return enabledProtocolsInput.split(","); + } + + private static String[] getCipherSuites(Configuration config) { + String cipherSuitesInput = config.get(TLS_CIPHER_SUITES); + if (cipherSuitesInput == null) { + return getDefaultCipherSuites(); + } else { + return cipherSuitesInput.split(","); + } + } + + /** + * Enable certificate file reloading by creating FileWatchers for keystore and truststore. + * AtomicReferences will be set with the new instances. resetContext - if not null - will be + * called when the file has been modified. + * @param keystoreWatcher Reference to keystoreFileWatcher. + * @param trustStoreWatcher Reference to truststoreFileWatcher. + * @param resetContext Callback for file changes. + */ + public static void enableCertFileReloading(Configuration config, + AtomicReference keystoreWatcher, + AtomicReference trustStoreWatcher, Runnable resetContext) + throws IOException { + String keyStoreLocation = config.get(TLS_CONFIG_KEYSTORE_LOCATION, ""); + keystoreWatcher.set(newFileChangeWatcher(keyStoreLocation, resetContext)); + String trustStoreLocation = config.get(TLS_CONFIG_TRUSTSTORE_LOCATION, ""); + trustStoreWatcher.set(newFileChangeWatcher(trustStoreLocation, resetContext)); + } + + private static FileChangeWatcher newFileChangeWatcher(String fileLocation, Runnable resetContext) + throws IOException { + if (fileLocation == null || fileLocation.isEmpty() || resetContext == null) { + return null; + } + final Path filePath = Paths.get(fileLocation).toAbsolutePath(); + Path parentPath = filePath.getParent(); + if (parentPath == null) { + throw new IOException("Key/trust store path does not have a parent: " + filePath); + } + FileChangeWatcher fileChangeWatcher = new FileChangeWatcher(parentPath, watchEvent -> { + handleWatchEvent(filePath, watchEvent, resetContext); + }); + fileChangeWatcher.start(); + return fileChangeWatcher; + } + + /** + * Handler for watch events that let us know a file we may care about has changed on disk. + * @param filePath the path to the file we are watching for changes. + * @param event the WatchEvent. + */ + private static void handleWatchEvent(Path filePath, WatchEvent event, Runnable resetContext) { + boolean shouldResetContext = false; + Path dirPath = filePath.getParent(); + if (event.kind().equals(StandardWatchEventKinds.OVERFLOW)) { + // If we get notified about possibly missed events, reload the key store / trust store just to + // be sure. + shouldResetContext = true; + } else if ( + event.kind().equals(StandardWatchEventKinds.ENTRY_MODIFY) + || event.kind().equals(StandardWatchEventKinds.ENTRY_CREATE) + ) { + Path eventFilePath = dirPath.resolve((Path) event.context()); + if (filePath.equals(eventFilePath)) { + shouldResetContext = true; + } + } + // Note: we don't care about delete events + if (shouldResetContext) { + LOG.info( + "Attempting to reset default SSL context after receiving watch event: {} with context: {}", + event.kind(), event.context()); + resetContext.run(); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring watch event and keeping previous default SSL context. " + + "Event kind: {} with context: {}", event.kind(), event.context()); + } + } + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestFileChangeWatcher.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestFileChangeWatcher.java new file mode 100644 index 000000000000..0d94a550d792 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestFileChangeWatcher.java @@ -0,0 +1,279 @@ +/* + * 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.io; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@Category({ IOTests.class, SmallTests.class }) +public class TestFileChangeWatcher { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFileChangeWatcher.class); + + private static File tempDir; + private static File tempFile; + + private static final Logger LOG = LoggerFactory.getLogger(TestFileChangeWatcher.class); + private static final HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility(); + + private static final long FS_TIMEOUT = 30000L; + + @BeforeClass + public static void createTempFile() throws IOException { + tempDir = new File(UTIL.getDataTestDir(TestFileChangeWatcher.class.getSimpleName()).toString()) + .getCanonicalFile(); + FileUtils.forceMkdir(tempDir); + tempFile = File.createTempFile("zk_test_", "", tempDir); + } + + @AfterClass + public static void cleanupTempDir() { + UTIL.cleanupTestDir(); + } + + @Test + public void testCallbackWorksOnFileChanges() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher(tempDir.toPath(), event -> { + LOG.info("Got an update: {} {}", event.kind(), event.context()); + // Filter out the extra ENTRY_CREATE events that are + // sometimes seen at the start. Even though we create the watcher + // after the file exists, sometimes we still get a create event. + if (StandardWatchEventKinds.ENTRY_CREATE.equals(event.kind())) { + return; + } + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // TODO hack + for (int i = 0; i < 3; i++) { + LOG.info("Modifying file, attempt {}", (i + 1)); + FileUtils.writeStringToFile(tempFile, "Hello world " + i + "\n", StandardCharsets.UTF_8, + true); + synchronized (events) { + if (events.size() < i + 1) { + events.wait(FS_TIMEOUT); + } + assertEquals("Wrong number of events", i + 1, events.size()); + WatchEvent event = events.get(i); + assertEquals(StandardWatchEventKinds.ENTRY_MODIFY, event.kind()); + assertEquals(tempFile.getName(), event.context().toString()); + } + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackWorksOnFileTouched() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher(tempDir.toPath(), event -> { + LOG.info("Got an update: {} {}", event.kind(), event.context()); + // Filter out the extra ENTRY_CREATE events that are + // sometimes seen at the start. Even though we create the watcher + // after the file exists, sometimes we still get a create event. + if (StandardWatchEventKinds.ENTRY_CREATE.equals(event.kind())) { + return; + } + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // TODO hack + LOG.info("Touching file"); + FileUtils.touch(tempFile); + synchronized (events) { + if (events.isEmpty()) { + events.wait(FS_TIMEOUT); + } + assertFalse(events.isEmpty()); + WatchEvent event = events.get(0); + assertEquals(StandardWatchEventKinds.ENTRY_MODIFY, event.kind()); + assertEquals(tempFile.getName(), event.context().toString()); + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackWorksOnFileAdded() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher(tempDir.toPath(), event -> { + LOG.info("Got an update: {} {}", event.kind(), event.context()); + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // TODO hack + File tempFile2 = File.createTempFile("zk_test_", "", tempDir); + tempFile2.deleteOnExit(); + synchronized (events) { + if (events.isEmpty()) { + events.wait(FS_TIMEOUT); + } + assertFalse(events.isEmpty()); + WatchEvent event = events.get(0); + assertEquals(StandardWatchEventKinds.ENTRY_CREATE, event.kind()); + assertEquals(tempFile2.getName(), event.context().toString()); + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackWorksOnFileDeleted() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher(tempDir.toPath(), event -> { + LOG.info("Got an update: {} {}", event.kind(), event.context()); + // Filter out the extra ENTRY_CREATE events that are + // sometimes seen at the start. Even though we create the watcher + // after the file exists, sometimes we still get a create event. + if (StandardWatchEventKinds.ENTRY_CREATE.equals(event.kind())) { + return; + } + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // TODO hack + tempFile.delete(); + synchronized (events) { + if (events.isEmpty()) { + events.wait(FS_TIMEOUT); + } + assertFalse(events.isEmpty()); + WatchEvent event = events.get(0); + assertEquals(StandardWatchEventKinds.ENTRY_DELETE, event.kind()); + assertEquals(tempFile.getName(), event.context().toString()); + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackErrorDoesNotCrashWatcherThread() + throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final AtomicInteger callCount = new AtomicInteger(0); + watcher = new FileChangeWatcher(tempDir.toPath(), event -> { + LOG.info("Got an update: {} {}", event.kind(), event.context()); + int oldValue; + synchronized (callCount) { + oldValue = callCount.getAndIncrement(); + callCount.notifyAll(); + } + if (oldValue == 0) { + throw new RuntimeException("This error should not crash the watcher thread"); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // TODO hack + LOG.info("Modifying file"); + FileUtils.writeStringToFile(tempFile, "Hello world\n", StandardCharsets.UTF_8, true); + synchronized (callCount) { + while (callCount.get() == 0) { + callCount.wait(FS_TIMEOUT); + } + } + LOG.info("Modifying file again"); + FileUtils.writeStringToFile(tempFile, "Hello world again\n", StandardCharsets.UTF_8, true); + synchronized (callCount) { + if (callCount.get() == 1) { + callCount.wait(FS_TIMEOUT); + } + } + // The value of callCount can exceed 1 only if the callback thread + // survives the exception thrown by the first callback. + assertTrue(callCount.get() > 1); + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/AbstractTestX509Parameterized.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/AbstractTestX509Parameterized.java new file mode 100644 index 000000000000..c2a9743fc6f4 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/AbstractTestX509Parameterized.java @@ -0,0 +1,128 @@ +/* + * 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.io.crypto.tls; + +import java.io.File; +import java.io.IOException; +import java.security.Security; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.runners.Parameterized; + +/** + * Base class for parameterized unit tests that use X509TestContext for testing different X509 + * parameter combinations (CA key type, cert key type, with/without a password, with/without + * hostname verification, etc). + *

+ * This base class takes care of setting up / cleaning up the test environment, and caching the + * X509TestContext objects used by the tests. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +public abstract class AbstractTestX509Parameterized { + + private static final HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility(); + private static X509TestContextProvider PROVIDER; + + @Parameterized.Parameter() + public X509KeyType caKeyType; + + @Parameterized.Parameter(value = 1) + public X509KeyType certKeyType; + + @Parameterized.Parameter(value = 2) + public char[] keyPassword; + + @Parameterized.Parameter(value = 3) + public Integer paramIndex; + + /** + * Default parameters suitable for most subclasses. See example usage in {@link TestX509Util}. + * @return an array of parameter combinations to test with. + */ + @Parameterized.Parameters( + name = "{index}: caKeyType={0}, certKeyType={1}, keyPassword={2}, paramIndex={3}") + public static Collection defaultParams() { + List result = new ArrayList<>(); + int paramIndex = 0; + for (X509KeyType caKeyType : X509KeyType.values()) { + for (X509KeyType certKeyType : X509KeyType.values()) { + for (char[] keyPassword : new char[][] { "".toCharArray(), "pa$$w0rd".toCharArray() }) { + result.add(new Object[] { caKeyType, certKeyType, keyPassword, paramIndex++ }); + } + } + } + return result; + } + + /** + * Because key generation and writing / deleting files is kind of expensive, we cache the certs + * and on-disk files between test cases. None of the test cases modify any of this data so it's + * safe to reuse between tests. This caching makes all test cases after the first one for a given + * parameter combination complete almost instantly. + */ + protected static Configuration conf; + + protected X509TestContext x509TestContext; + + @BeforeClass + public static void setUpBaseClass() throws Exception { + Security.addProvider(new BouncyCastleProvider()); + File dir = new File(UTIL.getDataTestDir(TestX509Util.class.getSimpleName()).toString()) + .getCanonicalFile(); + FileUtils.forceMkdir(dir); + PROVIDER = new X509TestContextProvider(UTIL.getConfiguration(), dir); + } + + @AfterClass + public static void cleanUpBaseClass() { + Security.removeProvider(BouncyCastleProvider.PROVIDER_NAME); + UTIL.cleanupTestDir(); + } + + @Before + public void setUp() throws IOException { + x509TestContext = PROVIDER.get(caKeyType, certKeyType, keyPassword); + x509TestContext.setConfigurations(KeyStoreFileType.JKS, KeyStoreFileType.JKS); + conf = new Configuration(UTIL.getConfiguration()); + } + + @After + public void cleanUp() { + x509TestContext.clearConfigurations(); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_OCSP); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_CLR); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_PROTOCOL); + System.clearProperty("com.sun.net.ssl.checkRevocation"); + System.clearProperty("com.sun.security.enableCRLDP"); + Security.setProperty("ocsp.enable", Boolean.FALSE.toString()); + Security.setProperty("com.sun.security.enableCRLDP", Boolean.FALSE.toString()); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestBCFKSFileLoader.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestBCFKSFileLoader.java new file mode 100644 index 000000000000..060c60a7a0ca --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestBCFKSFileLoader.java @@ -0,0 +1,118 @@ +/* + * 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.io.crypto.tls; + +import java.io.IOException; +import java.security.KeyStore; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@RunWith(Parameterized.class) +@Category({ SecurityTests.class, SmallTests.class }) +public class TestBCFKSFileLoader extends AbstractTestX509Parameterized { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBCFKSFileLoader.class); + + @Test + public void testLoadKeyStore() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.BCFKS).getAbsolutePath(); + KeyStore ks = new BCFKSFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + Assert.assertEquals(1, ks.size()); + } + + @Test(expected = Exception.class) + public void testLoadKeyStoreWithWrongPassword() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.BCFKS).getAbsolutePath(); + new BCFKSFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword("wrong password".toCharArray()).build().loadKeyStore(); + } + + @Test(expected = IOException.class) + public void testLoadKeyStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.BCFKS).getAbsolutePath(); + new BCFKSFileLoader.Builder().setKeyStorePath(path + ".does_not_exist") + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadKeyStoreWithNullFilePath() throws Exception { + new BCFKSFileLoader.Builder().setKeyStorePassword(x509TestContext.getKeyStorePassword()).build() + .loadKeyStore(); + } + + @Test(expected = IOException.class) + public void testLoadKeyStoreWithWrongFileType() throws Exception { + // Trying to load a PEM file with BCFKS loader should fail + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new BCFKSFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test + public void testLoadTrustStore() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.BCFKS).getAbsolutePath(); + KeyStore ts = new BCFKSFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + Assert.assertEquals(1, ts.size()); + } + + @Test(expected = Exception.class) + public void testLoadTrustStoreWithWrongPassword() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.BCFKS).getAbsolutePath(); + new BCFKSFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword("wrong password".toCharArray()).build().loadTrustStore(); + } + + @Test(expected = IOException.class) + public void testLoadTrustStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.BCFKS).getAbsolutePath(); + new BCFKSFileLoader.Builder().setTrustStorePath(path + ".does_not_exist") + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadTrustStoreWithNullFilePath() throws Exception { + new BCFKSFileLoader.Builder().setTrustStorePassword(x509TestContext.getTrustStorePassword()) + .build().loadTrustStore(); + } + + @Test(expected = IOException.class) + public void testLoadTrustStoreWithWrongFileType() throws Exception { + // Trying to load a PEM file with BCFKS loader should fail + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new BCFKSFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + } + +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestFileKeyStoreLoaderBuilderProvider.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestFileKeyStoreLoaderBuilderProvider.java new file mode 100644 index 000000000000..a80103483345 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestFileKeyStoreLoaderBuilderProvider.java @@ -0,0 +1,66 @@ +/* + * 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.io.crypto.tls; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@Category({ SecurityTests.class, SmallTests.class }) +public class TestFileKeyStoreLoaderBuilderProvider { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFileKeyStoreLoaderBuilderProvider.class); + + @Test + public void testGetBuilderForJKSFileType() { + FileKeyStoreLoader.Builder builder = + FileKeyStoreLoaderBuilderProvider.getBuilderForKeyStoreFileType(KeyStoreFileType.JKS); + Assert.assertTrue(builder instanceof JKSFileLoader.Builder); + } + + @Test + public void testGetBuilderForPEMFileType() { + FileKeyStoreLoader.Builder builder = + FileKeyStoreLoaderBuilderProvider.getBuilderForKeyStoreFileType(KeyStoreFileType.PEM); + Assert.assertTrue(builder instanceof PEMFileLoader.Builder); + } + + @Test + public void testGetBuilderForPKCS12FileType() { + FileKeyStoreLoader.Builder builder = + FileKeyStoreLoaderBuilderProvider.getBuilderForKeyStoreFileType(KeyStoreFileType.PKCS12); + Assert.assertTrue(builder instanceof PKCS12FileLoader.Builder); + } + + @Test(expected = NullPointerException.class) + public void testGetBuilderForNullFileType() { + FileKeyStoreLoaderBuilderProvider.getBuilderForKeyStoreFileType(null); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestHBaseHostnameVerifier.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestHBaseHostnameVerifier.java new file mode 100644 index 000000000000..12de61758360 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestHBaseHostnameVerifier.java @@ -0,0 +1,252 @@ +/* + * 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.io.crypto.tls; + +import static org.junit.Assert.fail; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.lang.invoke.MethodHandles; +import java.security.KeyPair; +import java.security.Security; +import java.security.cert.CertificateFactory; +import java.security.cert.X509Certificate; +import javax.net.ssl.SSLException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.bouncycastle.asn1.x500.X500Name; +import org.bouncycastle.asn1.x500.X500NameBuilder; +import org.bouncycastle.asn1.x500.style.BCStyle; +import org.bouncycastle.asn1.x509.GeneralName; +import org.bouncycastle.asn1.x509.GeneralNames; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.net.InetAddresses; + +/** + * Test cases taken and adapted from Apache ZooKeeper Project + * @see Base + * revision + */ +@Category({ MiscTests.class, SmallTests.class }) +public class TestHBaseHostnameVerifier { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseHostnameVerifier.class); + private static CertificateCreator certificateCreator; + private HBaseHostnameVerifier impl; + + @BeforeClass + public static void setupClass() throws Exception { + Security.addProvider(new BouncyCastleProvider()); + X500NameBuilder caNameBuilder = new X500NameBuilder(BCStyle.INSTANCE); + caNameBuilder.addRDN(BCStyle.CN, + MethodHandles.lookup().lookupClass().getCanonicalName() + " Root CA"); + KeyPair keyPair = X509TestHelpers.generateKeyPair(X509KeyType.EC); + X509Certificate caCert = X509TestHelpers.newSelfSignedCACert(caNameBuilder.build(), keyPair); + certificateCreator = new CertificateCreator(keyPair, caCert); + } + + @Before + public void setup() { + impl = new HBaseHostnameVerifier(); + } + + private static class CertificateCreator { + private final KeyPair caCertPair; + private final X509Certificate caCert; + + public CertificateCreator(KeyPair caCertPair, X509Certificate caCert) { + this.caCertPair = caCertPair; + this.caCert = caCert; + } + + public byte[] newCert(String cn, String... subjectAltName) throws Exception { + return X509TestHelpers.newCert(caCert, caCertPair, cn == null ? null : new X500Name(cn), + caCertPair.getPublic(), parseSubjectAltNames(subjectAltName)).getEncoded(); + } + + private GeneralNames parseSubjectAltNames(String... subjectAltName) { + if (subjectAltName == null || subjectAltName.length == 0) { + return null; + } + GeneralName[] names = new GeneralName[subjectAltName.length]; + for (int i = 0; i < subjectAltName.length; i++) { + String current = subjectAltName[i]; + int type; + if (InetAddresses.isInetAddress(current)) { + type = GeneralName.iPAddress; + } else if (current.startsWith("email:")) { + type = GeneralName.rfc822Name; + } else { + type = GeneralName.dNSName; + } + names[i] = new GeneralName(type, subjectAltName[i]); + } + return new GeneralNames(names); + } + + } + + @Test + public void testVerify() throws Exception { + final CertificateFactory cf = CertificateFactory.getInstance("X.509"); + InputStream in; + X509Certificate x509; + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=foo.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + + impl.verify("foo.com", x509); + exceptionPlease(impl, "a.foo.com", x509); + exceptionPlease(impl, "bar.com", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=\u82b1\u5b50.co.jp")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("\u82b1\u5b50.co.jp", x509); + exceptionPlease(impl, "a.\u82b1\u5b50.co.jp", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=foo.com", "bar.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + exceptionPlease(impl, "foo.com", x509); + exceptionPlease(impl, "a.foo.com", x509); + impl.verify("bar.com", x509); + exceptionPlease(impl, "a.bar.com", x509); + + in = new ByteArrayInputStream( + certificateCreator.newCert("CN=foo.com", "bar.com", "\u82b1\u5b50.co.jp")); + x509 = (X509Certificate) cf.generateCertificate(in); + exceptionPlease(impl, "foo.com", x509); + exceptionPlease(impl, "a.foo.com", x509); + impl.verify("bar.com", x509); + exceptionPlease(impl, "a.bar.com", x509); + + /* + * Java isn't extracting international subjectAlts properly. (Or OpenSSL isn't storing them + * properly). + */ + // DEFAULT.verify("\u82b1\u5b50.co.jp", x509 ); + // impl.verify("\u82b1\u5b50.co.jp", x509 ); + exceptionPlease(impl, "a.\u82b1\u5b50.co.jp", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=", "foo.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("foo.com", x509); + exceptionPlease(impl, "a.foo.com", x509); + + in = new ByteArrayInputStream( + certificateCreator.newCert("CN=foo.com, CN=bar.com, CN=\u82b1\u5b50.co.jp")); + x509 = (X509Certificate) cf.generateCertificate(in); + exceptionPlease(impl, "foo.com", x509); + exceptionPlease(impl, "a.foo.com", x509); + exceptionPlease(impl, "bar.com", x509); + exceptionPlease(impl, "a.bar.com", x509); + impl.verify("\u82b1\u5b50.co.jp", x509); + exceptionPlease(impl, "a.\u82b1\u5b50.co.jp", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=*.foo.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + exceptionPlease(impl, "foo.com", x509); + impl.verify("www.foo.com", x509); + impl.verify("\u82b1\u5b50.foo.com", x509); + exceptionPlease(impl, "a.b.foo.com", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=*.co.jp")); + x509 = (X509Certificate) cf.generateCertificate(in); + // Silly test because no-one would ever be able to lookup an IP address + // using "*.co.jp". + impl.verify("*.co.jp", x509); + impl.verify("foo.co.jp", x509); + impl.verify("\u82b1\u5b50.co.jp", x509); + + in = new ByteArrayInputStream( + certificateCreator.newCert("CN=*.foo.com", "*.bar.com", "*.\u82b1\u5b50.co.jp")); + x509 = (X509Certificate) cf.generateCertificate(in); + // try the foo.com variations + exceptionPlease(impl, "foo.com", x509); + exceptionPlease(impl, "www.foo.com", x509); + exceptionPlease(impl, "\u82b1\u5b50.foo.com", x509); + exceptionPlease(impl, "a.b.foo.com", x509); + // try the bar.com variations + exceptionPlease(impl, "bar.com", x509); + impl.verify("www.bar.com", x509); + impl.verify("\u82b1\u5b50.bar.com", x509); + exceptionPlease(impl, "a.b.bar.com", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=repository.infonotary.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("repository.infonotary.com", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=*.google.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("*.google.com", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=*.google.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("*.Google.com", x509); + + in = new ByteArrayInputStream(certificateCreator.newCert("CN=dummy-value.com", "1.1.1.1")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("1.1.1.1", x509); + + exceptionPlease(impl, "1.1.1.2", x509); + exceptionPlease(impl, "2001:0db8:85a3:0000:0000:8a2e:0370:1111", x509); + exceptionPlease(impl, "dummy-value.com", x509); + + in = new ByteArrayInputStream( + certificateCreator.newCert("CN=dummy-value.com", "2001:0db8:85a3:0000:0000:8a2e:0370:7334")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("2001:0db8:85a3:0000:0000:8a2e:0370:7334", x509); + + exceptionPlease(impl, "1.1.1.2", x509); + exceptionPlease(impl, "2001:0db8:85a3:0000:0000:8a2e:0370:1111", x509); + exceptionPlease(impl, "dummy-value.com", x509); + + in = new ByteArrayInputStream( + certificateCreator.newCert("CN=dummy-value.com", "2001:0db8:85a3:0000:0000:8a2e:0370:7334")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("2001:0db8:85a3:0000:0000:8a2e:0370:7334", x509); + impl.verify("[2001:0db8:85a3:0000:0000:8a2e:0370:7334]", x509); + + exceptionPlease(impl, "1.1.1.2", x509); + exceptionPlease(impl, "2001:0db8:85a3:0000:0000:8a2e:0370:1111", x509); + exceptionPlease(impl, "dummy-value.com", x509); + + in = new ByteArrayInputStream( + certificateCreator.newCert("CN=www.company.com", "email:email@example.com")); + x509 = (X509Certificate) cf.generateCertificate(in); + impl.verify("www.company.com", x509); + } + + private void exceptionPlease(final HBaseHostnameVerifier hv, final String host, + final X509Certificate x509) { + try { + hv.verify(host, x509); + fail("HostnameVerifier shouldn't allow [" + host + "]"); + } catch (final SSLException e) { + // whew! we're okay! + } + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestHBaseTrustManager.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestHBaseTrustManager.java new file mode 100644 index 000000000000..07fc87e01354 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestHBaseTrustManager.java @@ -0,0 +1,355 @@ +/* + * 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.io.crypto.tls; + +import static org.junit.Assert.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.math.BigInteger; +import java.net.InetAddress; +import java.net.Socket; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.Security; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Date; +import java.util.List; +import java.util.Random; +import javax.net.ssl.X509ExtendedTrustManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.bouncycastle.asn1.x500.X500NameBuilder; +import org.bouncycastle.asn1.x500.style.BCStyle; +import org.bouncycastle.asn1.x509.BasicConstraints; +import org.bouncycastle.asn1.x509.Extension; +import org.bouncycastle.asn1.x509.GeneralName; +import org.bouncycastle.asn1.x509.GeneralNames; +import org.bouncycastle.asn1.x509.KeyUsage; +import org.bouncycastle.cert.X509v3CertificateBuilder; +import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter; +import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.operator.ContentSigner; +import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.stubbing.Answer; + +// +/** + * Test cases taken and adapted from Apache ZooKeeper Project. We can only test calls to + * HBaseTrustManager using Sockets (not SSLEngines). This can be fine since the logic is the same. + * @see Base + * revision + */ +@Category({ MiscTests.class, SmallTests.class }) +public class TestHBaseTrustManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseTrustManager.class); + + private static KeyPair keyPair; + + private X509ExtendedTrustManager mockX509ExtendedTrustManager; + private static final String IP_ADDRESS = "127.0.0.1"; + private static final String HOSTNAME = "localhost"; + + private InetAddress mockInetAddressWithoutHostname; + private InetAddress mockInetAddressWithHostname; + private Socket mockSocketWithoutHostname; + private Socket mockSocketWithHostname; + + @BeforeClass + public static void createKeyPair() throws Exception { + Security.addProvider(new BouncyCastleProvider()); + KeyPairGenerator keyPairGenerator = + KeyPairGenerator.getInstance("RSA", BouncyCastleProvider.PROVIDER_NAME); + keyPairGenerator.initialize(4096); + keyPair = keyPairGenerator.genKeyPair(); + } + + @AfterClass + public static void removeBouncyCastleProvider() throws Exception { + Security.removeProvider("BC"); + } + + @Before + public void setup() throws Exception { + mockX509ExtendedTrustManager = mock(X509ExtendedTrustManager.class); + + mockInetAddressWithoutHostname = mock(InetAddress.class); + when(mockInetAddressWithoutHostname.getHostAddress()) + .thenAnswer((Answer) invocationOnMock -> IP_ADDRESS); + when(mockInetAddressWithoutHostname.toString()) + .thenAnswer((Answer) invocationOnMock -> "/" + IP_ADDRESS); + + mockInetAddressWithHostname = mock(InetAddress.class); + when(mockInetAddressWithHostname.getHostAddress()) + .thenAnswer((Answer) invocationOnMock -> IP_ADDRESS); + when(mockInetAddressWithHostname.getHostName()) + .thenAnswer((Answer) invocationOnMock -> HOSTNAME); + when(mockInetAddressWithHostname.toString()) + .thenAnswer((Answer) invocationOnMock -> HOSTNAME + "/" + IP_ADDRESS); + + mockSocketWithoutHostname = mock(Socket.class); + when(mockSocketWithoutHostname.getInetAddress()) + .thenAnswer((Answer) invocationOnMock -> mockInetAddressWithoutHostname); + + mockSocketWithHostname = mock(Socket.class); + when(mockSocketWithHostname.getInetAddress()) + .thenAnswer((Answer) invocationOnMock -> mockInetAddressWithHostname); + } + + @SuppressWarnings("JavaUtilDate") + private X509Certificate[] createSelfSignedCertificateChain(String ipAddress, String hostname) + throws Exception { + X500NameBuilder nameBuilder = new X500NameBuilder(BCStyle.INSTANCE); + nameBuilder.addRDN(BCStyle.CN, "NOT_LOCALHOST"); + Date notBefore = new Date(); + Calendar cal = Calendar.getInstance(); + cal.setTime(notBefore); + cal.add(Calendar.YEAR, 1); + Date notAfter = cal.getTime(); + BigInteger serialNumber = new BigInteger(128, new Random()); + + X509v3CertificateBuilder certificateBuilder = + new JcaX509v3CertificateBuilder(nameBuilder.build(), serialNumber, notBefore, notAfter, + nameBuilder.build(), keyPair.getPublic()) + .addExtension(Extension.basicConstraints, true, new BasicConstraints(0)) + .addExtension(Extension.keyUsage, true, + new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyCertSign | KeyUsage.cRLSign)); + + List generalNames = new ArrayList<>(); + if (ipAddress != null) { + generalNames.add(new GeneralName(GeneralName.iPAddress, ipAddress)); + } + if (hostname != null) { + generalNames.add(new GeneralName(GeneralName.dNSName, hostname)); + } + + if (!generalNames.isEmpty()) { + certificateBuilder.addExtension(Extension.subjectAlternativeName, true, + new GeneralNames(generalNames.toArray(new GeneralName[] {}))); + } + + ContentSigner contentSigner = + new JcaContentSignerBuilder("SHA256WithRSAEncryption").build(keyPair.getPrivate()); + + return new X509Certificate[] { + new JcaX509CertificateConverter().getCertificate(certificateBuilder.build(contentSigner)) }; + } + + @Test + public void testServerHostnameVerificationWithHostnameVerificationDisabled() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, false, false); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(IP_ADDRESS, HOSTNAME); + trustManager.checkServerTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(0)).getHostAddress(); + verify(mockInetAddressWithHostname, times(0)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @SuppressWarnings("checkstyle:linelength") + @Test + public void testServerTrustedWithHostnameVerificationDisabled() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, false, false); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(IP_ADDRESS, HOSTNAME); + trustManager.checkServerTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(0)).getHostAddress(); + verify(mockInetAddressWithHostname, times(0)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @Test + public void testServerTrustedWithHostnameVerificationEnabled() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, true); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(null, HOSTNAME); + trustManager.checkServerTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithHostname, times(1)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @Test + public void testServerTrustedWithHostnameVerificationEnabledUsingIpAddress() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, true); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(IP_ADDRESS, null); + trustManager.checkServerTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithHostname, times(0)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @Test + public void testServerTrustedWithHostnameVerificationEnabledNoReverseLookup() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, false); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(null, HOSTNAME); + + // We only include hostname in the cert above, but the socket passed in is for an ip address. + // This mismatch would succeed if reverse lookup is enabled, but here fails since it's + // not enabled. + assertThrows(CertificateException.class, + () -> trustManager.checkServerTrusted(certificateChain, null, mockSocketWithoutHostname)); + + verify(mockInetAddressWithoutHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithoutHostname, times(0)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, + mockSocketWithoutHostname); + } + + @Test + public void testServerTrustedWithHostnameVerificationEnabledWithHostnameNoReverseLookup() + throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, false); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(null, HOSTNAME); + + // since the socket inetAddress already has a hostname, we don't need reverse lookup. + // so this succeeds + trustManager.checkServerTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithHostname, times(1)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @Test + public void testClientTrustedWithHostnameVerificationDisabled() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, false, false); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(null, HOSTNAME); + trustManager.checkClientTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(0)).getHostAddress(); + verify(mockInetAddressWithHostname, times(0)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @Test + public void testClientTrustedWithHostnameVerificationEnabled() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, true); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(null, HOSTNAME); + trustManager.checkClientTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithHostname, times(1)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @Test + public void testClientTrustedWithHostnameVerificationEnabledUsingIpAddress() throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, true); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(IP_ADDRESS, null); + trustManager.checkClientTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithHostname, times(0)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, + mockSocketWithHostname); + } + + @Test + public void testClientTrustedWithHostnameVerificationEnabledWithoutReverseLookup() + throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, false); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(null, HOSTNAME); + + // We only include hostname in the cert above, but the socket passed in is for an ip address. + // This mismatch would succeed if reverse lookup is enabled, but here fails since it's + // not enabled. + assertThrows(CertificateException.class, + () -> trustManager.checkClientTrusted(certificateChain, null, mockSocketWithoutHostname)); + + verify(mockInetAddressWithoutHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithoutHostname, times(0)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, + mockSocketWithoutHostname); + } + + @Test + public void testClientTrustedWithHostnameVerificationEnabledWithHostnameNoReverseLookup() + throws Exception { + HBaseTrustManager trustManager = + new HBaseTrustManager(mockX509ExtendedTrustManager, true, false); + + X509Certificate[] certificateChain = createSelfSignedCertificateChain(null, HOSTNAME); + + // since the socket inetAddress already has a hostname, we don't need reverse lookup. + // so this succeeds + trustManager.checkClientTrusted(certificateChain, null, mockSocketWithHostname); + + verify(mockInetAddressWithHostname, times(1)).getHostAddress(); + verify(mockInetAddressWithHostname, times(1)).getHostName(); + + verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, + mockSocketWithHostname); + } + +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestJKSFileLoader.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestJKSFileLoader.java new file mode 100644 index 000000000000..6640e3b22f98 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestJKSFileLoader.java @@ -0,0 +1,117 @@ +/* + * 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.io.crypto.tls; + +import java.io.IOException; +import java.security.KeyStore; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@RunWith(Parameterized.class) +@Category({ SecurityTests.class, SmallTests.class }) +public class TestJKSFileLoader extends AbstractTestX509Parameterized { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJKSFileLoader.class); + + @Test + public void testLoadKeyStore() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + KeyStore ks = new JKSFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + Assert.assertEquals(1, ks.size()); + } + + @Test(expected = Exception.class) + public void testLoadKeyStoreWithWrongPassword() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + new JKSFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword("wrong password".toCharArray()).build().loadKeyStore(); + } + + @Test(expected = IOException.class) + public void testLoadKeyStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + new JKSFileLoader.Builder().setKeyStorePath(path + ".does_not_exist") + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadKeyStoreWithNullFilePath() throws Exception { + new JKSFileLoader.Builder().setKeyStorePassword(x509TestContext.getKeyStorePassword()).build() + .loadKeyStore(); + } + + @Test(expected = IOException.class) + public void testLoadKeyStoreWithWrongFileType() throws Exception { + // Trying to load a PEM file with JKS loader should fail + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new JKSFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test + public void testLoadTrustStore() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + KeyStore ts = new JKSFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + Assert.assertEquals(1, ts.size()); + } + + @Test(expected = Exception.class) + public void testLoadTrustStoreWithWrongPassword() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + new JKSFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword("wrong password".toCharArray()).build().loadTrustStore(); + } + + @Test(expected = IOException.class) + public void testLoadTrustStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + new JKSFileLoader.Builder().setTrustStorePath(path + ".does_not_exist") + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadTrustStoreWithNullFilePath() throws Exception { + new JKSFileLoader.Builder().setTrustStorePassword(x509TestContext.getTrustStorePassword()) + .build().loadTrustStore(); + } + + @Test(expected = IOException.class) + public void testLoadTrustStoreWithWrongFileType() throws Exception { + // Trying to load a PEM file with JKS loader should fail + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new JKSFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestKeyStoreFileType.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestKeyStoreFileType.java new file mode 100644 index 000000000000..d3f457fb4d33 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestKeyStoreFileType.java @@ -0,0 +1,120 @@ +/* + * 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.io.crypto.tls; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@Category({ SecurityTests.class, SmallTests.class }) +public class TestKeyStoreFileType { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyStoreFileType.class); + + @Test + public void testGetPropertyValue() { + Assert.assertEquals("PEM", KeyStoreFileType.PEM.getPropertyValue()); + Assert.assertEquals("JKS", KeyStoreFileType.JKS.getPropertyValue()); + Assert.assertEquals("PKCS12", KeyStoreFileType.PKCS12.getPropertyValue()); + Assert.assertEquals("BCFKS", KeyStoreFileType.BCFKS.getPropertyValue()); + } + + @Test + public void testFromPropertyValue() { + Assert.assertEquals(KeyStoreFileType.PEM, KeyStoreFileType.fromPropertyValue("PEM")); + Assert.assertEquals(KeyStoreFileType.JKS, KeyStoreFileType.fromPropertyValue("JKS")); + Assert.assertEquals(KeyStoreFileType.PKCS12, KeyStoreFileType.fromPropertyValue("PKCS12")); + Assert.assertEquals(KeyStoreFileType.BCFKS, KeyStoreFileType.fromPropertyValue("BCFKS")); + Assert.assertNull(KeyStoreFileType.fromPropertyValue("")); + Assert.assertNull(KeyStoreFileType.fromPropertyValue(null)); + } + + @Test + public void testFromPropertyValueIgnoresCase() { + Assert.assertEquals(KeyStoreFileType.PEM, KeyStoreFileType.fromPropertyValue("pem")); + Assert.assertEquals(KeyStoreFileType.JKS, KeyStoreFileType.fromPropertyValue("jks")); + Assert.assertEquals(KeyStoreFileType.PKCS12, KeyStoreFileType.fromPropertyValue("pkcs12")); + Assert.assertEquals(KeyStoreFileType.BCFKS, KeyStoreFileType.fromPropertyValue("bcfks")); + Assert.assertNull(KeyStoreFileType.fromPropertyValue("")); + Assert.assertNull(KeyStoreFileType.fromPropertyValue(null)); + } + + @Test(expected = IllegalArgumentException.class) + public void testFromPropertyValueThrowsOnBadPropertyValue() { + KeyStoreFileType.fromPropertyValue("foobar"); + } + + @Test + public void testFromFilename() { + Assert.assertEquals(KeyStoreFileType.JKS, KeyStoreFileType.fromFilename("mykey.jks")); + Assert.assertEquals(KeyStoreFileType.JKS, + KeyStoreFileType.fromFilename("/path/to/key/dir/mykey.jks")); + Assert.assertEquals(KeyStoreFileType.PEM, KeyStoreFileType.fromFilename("mykey.pem")); + Assert.assertEquals(KeyStoreFileType.PEM, + KeyStoreFileType.fromFilename("/path/to/key/dir/mykey.pem")); + Assert.assertEquals(KeyStoreFileType.PKCS12, KeyStoreFileType.fromFilename("mykey.p12")); + Assert.assertEquals(KeyStoreFileType.PKCS12, + KeyStoreFileType.fromFilename("/path/to/key/dir/mykey.p12")); + Assert.assertEquals(KeyStoreFileType.BCFKS, KeyStoreFileType.fromFilename("mykey.bcfks")); + Assert.assertEquals(KeyStoreFileType.BCFKS, + KeyStoreFileType.fromFilename("/path/to/key/dir/mykey.bcfks")); + } + + @Test(expected = IllegalArgumentException.class) + public void testFromFilenameThrowsOnBadFileExtension() { + KeyStoreFileType.fromFilename("prod.key"); + } + + @Test + public void testFromPropertyValueOrFileName() { + // Property value takes precedence if provided + Assert.assertEquals(KeyStoreFileType.JKS, + KeyStoreFileType.fromPropertyValueOrFileName("JKS", "prod.key")); + Assert.assertEquals(KeyStoreFileType.PEM, + KeyStoreFileType.fromPropertyValueOrFileName("PEM", "prod.key")); + Assert.assertEquals(KeyStoreFileType.PKCS12, + KeyStoreFileType.fromPropertyValueOrFileName("PKCS12", "prod.key")); + Assert.assertEquals(KeyStoreFileType.BCFKS, + KeyStoreFileType.fromPropertyValueOrFileName("BCFKS", "prod.key")); + // Falls back to filename detection if no property value + Assert.assertEquals(KeyStoreFileType.JKS, + KeyStoreFileType.fromPropertyValueOrFileName("", "prod.jks")); + } + + @Test(expected = IllegalArgumentException.class) + public void testFromPropertyValueOrFileNameThrowsOnBadPropertyValue() { + KeyStoreFileType.fromPropertyValueOrFileName("foobar", "prod.jks"); + } + + @Test(expected = IllegalArgumentException.class) + public void testFromPropertyValueOrFileNameThrowsOnBadFileExtension() { + KeyStoreFileType.fromPropertyValueOrFileName("", "prod.key"); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestPEMFileLoader.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestPEMFileLoader.java new file mode 100644 index 000000000000..0c9924f09075 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestPEMFileLoader.java @@ -0,0 +1,112 @@ +/* + * 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.io.crypto.tls; + +import java.io.IOException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@RunWith(Parameterized.class) +@Category({ SecurityTests.class, SmallTests.class }) +public class TestPEMFileLoader extends AbstractTestX509Parameterized { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPEMFileLoader.class); + + @Test + public void testLoadKeyStore() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + KeyStore ks = new PEMFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + Assert.assertEquals(1, ks.size()); + } + + @Test(expected = Exception.class) + public void testLoadKeyStoreWithWrongPassword() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new PEMFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword("wrong password".toCharArray()).build().loadKeyStore(); + } + + @Test(expected = IOException.class) + public void testLoadKeyStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new PEMFileLoader.Builder().setKeyStorePath(path + ".does_not_exist") + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadKeyStoreWithNullFilePath() throws Exception { + new PEMFileLoader.Builder().setKeyStorePassword(x509TestContext.getKeyStorePassword()).build() + .loadKeyStore(); + } + + @Test(expected = KeyStoreException.class) + public void testLoadKeyStoreWithWrongFileType() throws Exception { + // Trying to load a JKS file with PEM loader should fail + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + new PEMFileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test + public void testLoadTrustStore() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + KeyStore ts = new PEMFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + Assert.assertEquals(1, ts.size()); + } + + @Test(expected = IOException.class) + public void testLoadTrustStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new PEMFileLoader.Builder().setTrustStorePath(path + ".does_not_exist") + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadTrustStoreWithNullFilePath() throws Exception { + new PEMFileLoader.Builder().setTrustStorePassword(x509TestContext.getTrustStorePassword()) + .build().loadTrustStore(); + } + + @Test + public void testLoadTrustStoreWithWrongFileType() throws Exception { + // Trying to load a JKS file with PEM loader should fail + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(); + KeyStore ts = new PEMFileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + Assert.assertEquals(0, ts.size()); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestPKCS12FileLoader.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestPKCS12FileLoader.java new file mode 100644 index 000000000000..a0ff83833e22 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestPKCS12FileLoader.java @@ -0,0 +1,117 @@ +/* + * 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.io.crypto.tls; + +import java.io.IOException; +import java.security.KeyStore; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@RunWith(Parameterized.class) +@Category({ SecurityTests.class, SmallTests.class }) +public class TestPKCS12FileLoader extends AbstractTestX509Parameterized { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPKCS12FileLoader.class); + + @Test + public void testLoadKeyStore() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(); + KeyStore ks = new PKCS12FileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + Assert.assertEquals(1, ks.size()); + } + + @Test(expected = Exception.class) + public void testLoadKeyStoreWithWrongPassword() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(); + new PKCS12FileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword("wrong password".toCharArray()).build().loadKeyStore(); + } + + @Test(expected = IOException.class) + public void testLoadKeyStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(); + new PKCS12FileLoader.Builder().setKeyStorePath(path + ".does_not_exist") + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadKeyStoreWithNullFilePath() throws Exception { + new PKCS12FileLoader.Builder().setKeyStorePassword(x509TestContext.getKeyStorePassword()) + .build().loadKeyStore(); + } + + @Test(expected = IOException.class) + public void testLoadKeyStoreWithWrongFileType() throws Exception { + // Trying to load a PEM file with PKCS12 loader should fail + String path = x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new PKCS12FileLoader.Builder().setKeyStorePath(path) + .setKeyStorePassword(x509TestContext.getKeyStorePassword()).build().loadKeyStore(); + } + + @Test + public void testLoadTrustStore() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(); + KeyStore ts = new PKCS12FileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + Assert.assertEquals(1, ts.size()); + } + + @Test(expected = Exception.class) + public void testLoadTrustStoreWithWrongPassword() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(); + new PKCS12FileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword("wrong password".toCharArray()).build().loadTrustStore(); + } + + @Test(expected = IOException.class) + public void testLoadTrustStoreWithWrongFilePath() throws Exception { + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(); + new PKCS12FileLoader.Builder().setTrustStorePath(path + ".does_not_exist") + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + } + + @Test(expected = NullPointerException.class) + public void testLoadTrustStoreWithNullFilePath() throws Exception { + new PKCS12FileLoader.Builder().setTrustStorePassword(x509TestContext.getTrustStorePassword()) + .build().loadTrustStore(); + } + + @Test(expected = IOException.class) + public void testLoadTrustStoreWithWrongFileType() throws Exception { + // Trying to load a PEM file with PKCS12 loader should fail + String path = x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(); + new PKCS12FileLoader.Builder().setTrustStorePath(path) + .setTrustStorePassword(x509TestContext.getTrustStorePassword()).build().loadTrustStore(); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestX509Util.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestX509Util.java new file mode 100644 index 000000000000..dd81403af6f2 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/TestX509Util.java @@ -0,0 +1,412 @@ +/* + * 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.io.crypto.tls; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeThat; +import static org.mockito.Mockito.mock; + +import java.security.Security; +import java.util.Arrays; +import java.util.Collections; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.exceptions.KeyManagerException; +import org.apache.hadoop.hbase.exceptions.SSLContextException; +import org.apache.hadoop.hbase.exceptions.TrustManagerException; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufAllocator; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslContext; + +/** + * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@RunWith(Parameterized.class) +@Category({ SecurityTests.class, SmallTests.class }) +public class TestX509Util extends AbstractTestX509Parameterized { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestX509Util.class); + + private static final char[] EMPTY_CHAR_ARRAY = new char[0]; + + @Test + public void testCreateSSLContextWithClientAuthDefault() throws Exception { + SslContext sslContext = X509Util.createSslContextForServer(conf); + ByteBufAllocator byteBufAllocatorMock = mock(ByteBufAllocator.class); + assertTrue(sslContext.newEngine(byteBufAllocatorMock).getNeedClientAuth()); + } + + @Test + public void testCreateSSLContextWithClientAuthNEED() throws Exception { + conf.set(X509Util.HBASE_SERVER_NETTY_TLS_CLIENT_AUTH_MODE, X509Util.ClientAuth.NEED.name()); + SslContext sslContext = X509Util.createSslContextForServer(conf); + ByteBufAllocator byteBufAllocatorMock = mock(ByteBufAllocator.class); + assertTrue(sslContext.newEngine(byteBufAllocatorMock).getNeedClientAuth()); + } + + @Test + public void testCreateSSLContextWithClientAuthWANT() throws Exception { + conf.set(X509Util.HBASE_SERVER_NETTY_TLS_CLIENT_AUTH_MODE, X509Util.ClientAuth.WANT.name()); + SslContext sslContext = X509Util.createSslContextForServer(conf); + ByteBufAllocator byteBufAllocatorMock = mock(ByteBufAllocator.class); + assertTrue(sslContext.newEngine(byteBufAllocatorMock).getWantClientAuth()); + } + + @Test + public void testCreateSSLContextWithClientAuthNONE() throws Exception { + conf.set(X509Util.HBASE_SERVER_NETTY_TLS_CLIENT_AUTH_MODE, X509Util.ClientAuth.NONE.name()); + SslContext sslContext = X509Util.createSslContextForServer(conf); + ByteBufAllocator byteBufAllocatorMock = mock(ByteBufAllocator.class); + assertFalse(sslContext.newEngine(byteBufAllocatorMock).getNeedClientAuth()); + assertFalse(sslContext.newEngine(byteBufAllocatorMock).getWantClientAuth()); + } + + @Test + public void testCreateSSLContextWithoutCustomProtocol() throws Exception { + SslContext sslContext = X509Util.createSslContextForClient(conf); + ByteBufAllocator byteBufAllocatorMock = mock(ByteBufAllocator.class); + assertArrayEquals(new String[] { X509Util.DEFAULT_PROTOCOL }, + sslContext.newEngine(byteBufAllocatorMock).getEnabledProtocols()); + } + + @Test + public void testCreateSSLContextWithCustomProtocol() throws Exception { + final String protocol = "TLSv1.1"; + conf.set(X509Util.TLS_CONFIG_PROTOCOL, protocol); + ByteBufAllocator byteBufAllocatorMock = mock(ByteBufAllocator.class); + SslContext sslContext = X509Util.createSslContextForServer(conf); + assertEquals(Collections.singletonList(protocol), + Arrays.asList(sslContext.newEngine(byteBufAllocatorMock).getEnabledProtocols())); + } + + @Test(expected = SSLContextException.class) + public void testCreateSSLContextWithoutKeyStoreLocationServer() throws Exception { + conf.unset(X509Util.TLS_CONFIG_KEYSTORE_LOCATION); + X509Util.createSslContextForServer(conf); + } + + @Test + public void testCreateSSLContextWithoutKeyStoreLocationClient() throws Exception { + conf.unset(X509Util.TLS_CONFIG_KEYSTORE_LOCATION); + X509Util.createSslContextForClient(conf); + } + + @Test + public void testCreateSSLContextWithoutTrustStoreLocationClient() throws Exception { + conf.unset(X509Util.TLS_CONFIG_TRUSTSTORE_LOCATION); + X509Util.createSslContextForClient(conf); + } + + @Test + public void testCreateSSLContextWithoutTrustStoreLocationServer() throws Exception { + conf.unset(X509Util.TLS_CONFIG_TRUSTSTORE_LOCATION); + X509Util.createSslContextForServer(conf); + } + + // It would be great to test the value of PKIXBuilderParameters#setRevocationEnabled, + // but it does not appear to be possible + @Test + public void testCRLEnabled() throws Exception { + conf.setBoolean(X509Util.TLS_CONFIG_CLR, true); + X509Util.createSslContextForServer(conf); + assertTrue(Boolean.valueOf(System.getProperty("com.sun.net.ssl.checkRevocation"))); + assertTrue(Boolean.valueOf(System.getProperty("com.sun.security.enableCRLDP"))); + assertFalse(Boolean.valueOf(Security.getProperty("ocsp.enable"))); + } + + @Test + public void testCRLDisabled() throws Exception { + X509Util.createSslContextForServer(conf); + assertFalse(Boolean.valueOf(System.getProperty("com.sun.net.ssl.checkRevocation"))); + assertFalse(Boolean.valueOf(System.getProperty("com.sun.security.enableCRLDP"))); + assertFalse(Boolean.valueOf(Security.getProperty("ocsp.enable"))); + } + + @Test + public void testLoadPEMKeyStore() throws Exception { + // Make sure we can instantiate a key manager from the PEM file on disk + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(), + x509TestContext.getKeyStorePassword(), KeyStoreFileType.PEM.getPropertyValue()); + } + + @Test + public void testLoadPEMKeyStoreNullPassword() throws Exception { + assumeThat(x509TestContext.getKeyStorePassword(), equalTo(EMPTY_CHAR_ARRAY)); + // Make sure that empty password and null password are treated the same + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(), null, + KeyStoreFileType.PEM.getPropertyValue()); + } + + @Test + public void testLoadPEMKeyStoreAutodetectStoreFileType() throws Exception { + // Make sure we can instantiate a key manager from the PEM file on disk + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(), + x509TestContext.getKeyStorePassword(), + null /* null StoreFileType means 'autodetect from file extension' */); + } + + @Test(expected = KeyManagerException.class) + public void testLoadPEMKeyStoreWithWrongPassword() throws Exception { + // Attempting to load with the wrong key password should fail + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PEM).getAbsolutePath(), + "wrong password".toCharArray(), // intentionally use the wrong password + KeyStoreFileType.PEM.getPropertyValue()); + } + + @Test + public void testLoadPEMTrustStore() throws Exception { + // Make sure we can instantiate a trust manager from the PEM file on disk + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(), + x509TestContext.getTrustStorePassword(), KeyStoreFileType.PEM.getPropertyValue(), false, + false, true, true); + } + + @Test + public void testLoadPEMTrustStoreNullPassword() throws Exception { + assumeThat(x509TestContext.getTrustStorePassword(), equalTo(EMPTY_CHAR_ARRAY)); + // Make sure that empty password and null password are treated the same + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(), null, + KeyStoreFileType.PEM.getPropertyValue(), false, false, true, true); + } + + @Test + public void testLoadPEMTrustStoreAutodetectStoreFileType() throws Exception { + // Make sure we can instantiate a trust manager from the PEM file on disk + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath(), + x509TestContext.getTrustStorePassword(), null, // null StoreFileType means 'autodetect from + // file extension' + false, false, true, true); + } + + @Test + public void testLoadJKSKeyStore() throws Exception { + // Make sure we can instantiate a key manager from the JKS file on disk + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), + x509TestContext.getKeyStorePassword(), KeyStoreFileType.JKS.getPropertyValue()); + } + + @Test + public void testLoadJKSKeyStoreNullPassword() throws Exception { + assumeThat(x509TestContext.getKeyStorePassword(), equalTo(EMPTY_CHAR_ARRAY)); + // Make sure that empty password and null password are treated the same + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), null, + KeyStoreFileType.JKS.getPropertyValue()); + } + + @Test + public void testLoadJKSKeyStoreAutodetectStoreFileType() throws Exception { + // Make sure we can instantiate a key manager from the JKS file on disk + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), + x509TestContext.getKeyStorePassword(), + null /* null StoreFileType means 'autodetect from file extension' */); + } + + @Test + public void testLoadJKSKeyStoreWithWrongPassword() { + assertThrows(KeyManagerException.class, () -> { + // Attempting to load with the wrong key password should fail + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), + "wrong password".toCharArray(), KeyStoreFileType.JKS.getPropertyValue()); + }); + } + + @Test + public void testLoadJKSTrustStore() throws Exception { + // Make sure we can instantiate a trust manager from the JKS file on disk + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), + x509TestContext.getTrustStorePassword(), KeyStoreFileType.JKS.getPropertyValue(), true, true, + true, true); + } + + @Test + public void testLoadJKSTrustStoreNullPassword() throws Exception { + assumeThat(x509TestContext.getTrustStorePassword(), equalTo(EMPTY_CHAR_ARRAY)); + // Make sure that empty password and null password are treated the same + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), null, + KeyStoreFileType.JKS.getPropertyValue(), false, false, true, true); + } + + @Test + public void testLoadJKSTrustStoreAutodetectStoreFileType() throws Exception { + // Make sure we can instantiate a trust manager from the JKS file on disk + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), + x509TestContext.getTrustStorePassword(), null, // null StoreFileType means 'autodetect from + // file extension' + true, true, true, true); + } + + @Test + public void testLoadJKSTrustStoreWithWrongPassword() { + assertThrows(TrustManagerException.class, () -> { + // Attempting to load with the wrong key password should fail + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.JKS).getAbsolutePath(), + "wrong password".toCharArray(), KeyStoreFileType.JKS.getPropertyValue(), true, true, true, + true); + }); + } + + @Test + public void testLoadPKCS12KeyStore() throws Exception { + // Make sure we can instantiate a key manager from the PKCS12 file on disk + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), + x509TestContext.getKeyStorePassword(), KeyStoreFileType.PKCS12.getPropertyValue()); + } + + @Test + public void testLoadPKCS12KeyStoreNullPassword() throws Exception { + assumeThat(x509TestContext.getKeyStorePassword(), equalTo(EMPTY_CHAR_ARRAY)); + // Make sure that empty password and null password are treated the same + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), null, + KeyStoreFileType.PKCS12.getPropertyValue()); + } + + @Test + public void testLoadPKCS12KeyStoreAutodetectStoreFileType() throws Exception { + // Make sure we can instantiate a key manager from the PKCS12 file on disk + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), + x509TestContext.getKeyStorePassword(), + null /* null StoreFileType means 'autodetect from file extension' */); + } + + @Test + public void testLoadPKCS12KeyStoreWithWrongPassword() { + assertThrows(KeyManagerException.class, () -> { + // Attempting to load with the wrong key password should fail + X509Util.createKeyManager( + x509TestContext.getKeyStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), + "wrong password".toCharArray(), KeyStoreFileType.PKCS12.getPropertyValue()); + }); + } + + @Test + public void testLoadPKCS12TrustStore() throws Exception { + // Make sure we can instantiate a trust manager from the PKCS12 file on disk + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), + x509TestContext.getTrustStorePassword(), KeyStoreFileType.PKCS12.getPropertyValue(), true, + true, true, true); + } + + @Test + public void testLoadPKCS12TrustStoreNullPassword() throws Exception { + assumeThat(x509TestContext.getTrustStorePassword(), equalTo(EMPTY_CHAR_ARRAY)); + // Make sure that empty password and null password are treated the same + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), null, + KeyStoreFileType.PKCS12.getPropertyValue(), false, false, true, true); + } + + @Test + public void testLoadPKCS12TrustStoreAutodetectStoreFileType() throws Exception { + // Make sure we can instantiate a trust manager from the PKCS12 file on disk + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), + x509TestContext.getTrustStorePassword(), null, // null StoreFileType means 'autodetect from + // file extension' + true, true, true, true); + } + + @Test + public void testLoadPKCS12TrustStoreWithWrongPassword() { + assertThrows(TrustManagerException.class, () -> { + // Attempting to load with the wrong key password should fail + X509Util.createTrustManager( + x509TestContext.getTrustStoreFile(KeyStoreFileType.PKCS12).getAbsolutePath(), + "wrong password".toCharArray(), KeyStoreFileType.PKCS12.getPropertyValue(), true, true, + true, true); + }); + } + + @Test + public void testGetDefaultCipherSuitesJava8() { + String[] cipherSuites = X509Util.getDefaultCipherSuitesForJavaVersion("1.8"); + // Java 8 default should have the CBC suites first + assertThat(cipherSuites[0], containsString("CBC")); + } + + @Test + public void testGetDefaultCipherSuitesJava9() { + String[] cipherSuites = X509Util.getDefaultCipherSuitesForJavaVersion("9"); + // Java 9+ default should have the GCM suites first + assertThat(cipherSuites[0], containsString("GCM")); + } + + @Test + public void testGetDefaultCipherSuitesJava10() { + String[] cipherSuites = X509Util.getDefaultCipherSuitesForJavaVersion("10"); + // Java 9+ default should have the GCM suites first + assertThat(cipherSuites[0], containsString("GCM")); + } + + @Test + public void testGetDefaultCipherSuitesJava11() { + String[] cipherSuites = X509Util.getDefaultCipherSuitesForJavaVersion("11"); + // Java 9+ default should have the GCM suites first + assertThat(cipherSuites[0], containsString("GCM")); + } + + @Test + public void testGetDefaultCipherSuitesUnknownVersion() { + String[] cipherSuites = X509Util.getDefaultCipherSuitesForJavaVersion("notaversion"); + // If version can't be parsed, use the more conservative Java 8 default + assertThat(cipherSuites[0], containsString("CBC")); + } + + @Test + public void testGetDefaultCipherSuitesNullVersion() { + assertThrows(NullPointerException.class, () -> { + X509Util.getDefaultCipherSuitesForJavaVersion(null); + }); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509KeyType.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509KeyType.java new file mode 100644 index 000000000000..1d5c042f04a6 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509KeyType.java @@ -0,0 +1,32 @@ +/* + * 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.io.crypto.tls; + +/** + * Represents a type of key pair used for X509 certs in tests. The two options are RSA or EC + * (elliptic curve). + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +public enum X509KeyType { + RSA, + EC +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestContext.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestContext.java new file mode 100644 index 000000000000..dd2975070f29 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestContext.java @@ -0,0 +1,612 @@ +/* + * 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.io.crypto.tls; + +import static java.util.Objects.requireNonNull; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.nio.charset.StandardCharsets; +import java.security.GeneralSecurityException; +import java.security.KeyPair; +import java.security.Security; +import java.security.cert.X509Certificate; +import java.util.Arrays; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.yetus.audience.InterfaceAudience; +import org.bouncycastle.asn1.x500.X500Name; +import org.bouncycastle.asn1.x500.X500NameBuilder; +import org.bouncycastle.asn1.x500.style.BCStyle; +import org.bouncycastle.asn1.x509.GeneralName; +import org.bouncycastle.asn1.x509.GeneralNames; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.operator.OperatorCreationException; + +/** + * This class simplifies the creation of certificates and private keys for SSL/TLS connections. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +public final class X509TestContext { + + private static final String TRUST_STORE_PREFIX = "hbase_test_ca"; + private static final String KEY_STORE_PREFIX = "hbase_test_key"; + + private final File tempDir; + private final Configuration conf; + + private X509Certificate trustStoreCertificate; + private final char[] trustStorePassword; + private KeyPair trustStoreKeyPair; + private File trustStoreJksFile; + private File trustStorePemFile; + private File trustStorePkcs12File; + private File trustStoreBcfksFile; + + private KeyPair keyStoreKeyPair; + private X509Certificate keyStoreCertificate; + private final char[] keyStorePassword; + private File keyStoreJksFile; + private File keyStorePemFile; + private File keyStorePkcs12File; + private File keyStoreBcfksFile; + + /** + * Constructor is intentionally private, use the Builder class instead. + * @param conf the configuration + * @param tempDir the directory in which key store and trust store temp files will be + * written. + * @param trustStoreKeyPair the key pair for the trust store. + * @param trustStorePassword the password to protect a JKS trust store (ignored for PEM trust + * stores). + * @param keyStoreKeyPair the key pair for the key store. + * @param keyStorePassword the password to protect the key store private key. + */ + private X509TestContext(Configuration conf, File tempDir, KeyPair trustStoreKeyPair, + char[] trustStorePassword, KeyPair keyStoreKeyPair, char[] keyStorePassword) + throws IOException, GeneralSecurityException, OperatorCreationException { + if (Security.getProvider(BouncyCastleProvider.PROVIDER_NAME) == null) { + throw new IllegalStateException("BC Security provider was not found"); + } + this.conf = conf; + this.tempDir = requireNonNull(tempDir); + if (!tempDir.isDirectory()) { + throw new IllegalArgumentException("Not a directory: " + tempDir); + } + + this.trustStoreKeyPair = trustStoreKeyPair; + this.trustStorePassword = requireNonNull(trustStorePassword); + this.keyStoreKeyPair = requireNonNull(keyStoreKeyPair); + this.keyStorePassword = requireNonNull(keyStorePassword); + + createCertificates(); + + trustStorePkcs12File = null; + trustStorePemFile = null; + trustStoreJksFile = null; + keyStorePkcs12File = null; + keyStorePemFile = null; + keyStoreJksFile = null; + } + + /** + * Used by {@link #cloneWithNewKeystoreCert(X509Certificate)}. Should set all fields except + * generated keystore path fields + */ + private X509TestContext(File tempDir, Configuration conf, X509Certificate trustStoreCertificate, + char[] trustStorePassword, KeyPair trustStoreKeyPair, File trustStoreJksFile, + File trustStorePemFile, File trustStorePkcs12File, KeyPair keyStoreKeyPair, + char[] keyStorePassword, X509Certificate keyStoreCertificate) { + this.tempDir = tempDir; + this.conf = conf; + this.trustStoreCertificate = trustStoreCertificate; + this.trustStorePassword = trustStorePassword; + this.trustStoreKeyPair = trustStoreKeyPair; + this.trustStoreJksFile = trustStoreJksFile; + this.trustStorePemFile = trustStorePemFile; + this.trustStorePkcs12File = trustStorePkcs12File; + this.keyStoreKeyPair = keyStoreKeyPair; + this.keyStoreCertificate = keyStoreCertificate; + this.keyStorePassword = keyStorePassword; + keyStorePkcs12File = null; + keyStorePemFile = null; + keyStoreJksFile = null; + } + + /** + * Generates a new certificate using this context's CA and keystoreKeyPair. By default, the cert + * will have localhost in the subjectAltNames. This can be overridden by passing one or more + * string arguments after the cert name. The expectation for those arguments is that they are + * valid DNS names. + */ + public X509Certificate newCert(X500Name name, String... subjectAltNames) + throws GeneralSecurityException, IOException, OperatorCreationException { + if (subjectAltNames.length == 0) { + return X509TestHelpers.newCert(trustStoreCertificate, trustStoreKeyPair, name, + keyStoreKeyPair.getPublic()); + } + GeneralName[] names = new GeneralName[subjectAltNames.length]; + for (int i = 0; i < subjectAltNames.length; i++) { + names[i] = new GeneralName(GeneralName.dNSName, subjectAltNames[i]); + } + return X509TestHelpers.newCert(trustStoreCertificate, trustStoreKeyPair, name, + keyStoreKeyPair.getPublic(), new GeneralNames(names)); + } + + public File getTempDir() { + return tempDir; + } + + public char[] getTrustStorePassword() { + return trustStorePassword; + } + + /** + * Returns the path to the trust store file in the given format (JKS or PEM). Note that the file + * is created lazily, the first time this method is called. The trust store file is temporary and + * will be deleted on exit. + * @param storeFileType the store file type (JKS or PEM). + * @return the path to the trust store file. + * @throws IOException if there is an error creating the trust store file. + */ + public File getTrustStoreFile(KeyStoreFileType storeFileType) throws IOException { + switch (storeFileType) { + case JKS: + return getTrustStoreJksFile(); + case PEM: + return getTrustStorePemFile(); + case PKCS12: + return getTrustStorePkcs12File(); + case BCFKS: + return getTrustStoreBcfksFile(); + default: + throw new IllegalArgumentException("Invalid trust store type: " + storeFileType + + ", must be one of: " + Arrays.toString(KeyStoreFileType.values())); + } + } + + private File getTrustStoreJksFile() throws IOException { + if (trustStoreJksFile == null) { + trustStoreJksFile = File.createTempFile(TRUST_STORE_PREFIX, + KeyStoreFileType.JKS.getDefaultFileExtension(), tempDir); + trustStoreJksFile.deleteOnExit(); + generateTrustStoreJksFile(); + } + return trustStoreJksFile; + } + + private void generateTrustStoreJksFile() throws IOException { + try (final FileOutputStream trustStoreOutputStream = new FileOutputStream(trustStoreJksFile)) { + byte[] bytes = + X509TestHelpers.certToJavaTrustStoreBytes(trustStoreCertificate, trustStorePassword); + trustStoreOutputStream.write(bytes); + trustStoreOutputStream.flush(); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + } + + private File getTrustStorePemFile() throws IOException { + if (trustStorePemFile == null) { + trustStorePemFile = File.createTempFile(TRUST_STORE_PREFIX, + KeyStoreFileType.PEM.getDefaultFileExtension(), tempDir); + trustStorePemFile.deleteOnExit(); + generateTrustStorePemFile(); + } + return trustStorePemFile; + } + + private void generateTrustStorePemFile() throws IOException { + FileUtils.writeStringToFile(trustStorePemFile, + X509TestHelpers.pemEncodeX509Certificate(trustStoreCertificate), StandardCharsets.US_ASCII, + false); + } + + private File getTrustStorePkcs12File() throws IOException { + if (trustStorePkcs12File == null) { + trustStorePkcs12File = File.createTempFile(TRUST_STORE_PREFIX, + KeyStoreFileType.PKCS12.getDefaultFileExtension(), tempDir); + trustStorePkcs12File.deleteOnExit(); + generateTrustStorePkcs12File(); + } + return trustStorePkcs12File; + } + + private void generateTrustStorePkcs12File() throws IOException { + try ( + final FileOutputStream trustStoreOutputStream = new FileOutputStream(trustStorePkcs12File)) { + byte[] bytes = + X509TestHelpers.certToPKCS12TrustStoreBytes(trustStoreCertificate, trustStorePassword); + trustStoreOutputStream.write(bytes); + trustStoreOutputStream.flush(); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + } + + private File getTrustStoreBcfksFile() throws IOException { + if (trustStoreBcfksFile == null) { + trustStoreBcfksFile = File.createTempFile(TRUST_STORE_PREFIX, + KeyStoreFileType.BCFKS.getDefaultFileExtension(), tempDir); + trustStoreBcfksFile.deleteOnExit(); + generateTrustStoreBcfksFile(); + } + return trustStoreBcfksFile; + } + + private void generateTrustStoreBcfksFile() throws IOException { + try ( + final FileOutputStream trustStoreOutputStream = new FileOutputStream(trustStoreBcfksFile)) { + byte[] bytes = + X509TestHelpers.certToBCFKSTrustStoreBytes(trustStoreCertificate, trustStorePassword); + trustStoreOutputStream.write(bytes); + trustStoreOutputStream.flush(); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + } + + public X509Certificate getKeyStoreCertificate() { + return keyStoreCertificate; + } + + public char[] getKeyStorePassword() { + return keyStorePassword; + } + + public boolean isKeyStoreEncrypted() { + return keyStorePassword != null; + } + + public Configuration getConf() { + return conf; + } + + /** + * Returns the path to the key store file in the given format (JKS, PEM, ...). Note that the file + * is created lazily, the first time this method is called. The key store file is temporary and + * will be deleted on exit. + * @param storeFileType the store file type (JKS, PEM, ...). + * @return the path to the key store file. + * @throws IOException if there is an error creating the key store file. + */ + public File getKeyStoreFile(KeyStoreFileType storeFileType) throws IOException { + switch (storeFileType) { + case JKS: + return getKeyStoreJksFile(); + case PEM: + return getKeyStorePemFile(); + case PKCS12: + return getKeyStorePkcs12File(); + case BCFKS: + return getKeyStoreBcfksFile(); + default: + throw new IllegalArgumentException("Invalid key store type: " + storeFileType + + ", must be one of: " + Arrays.toString(KeyStoreFileType.values())); + } + } + + private File getKeyStoreJksFile() throws IOException { + if (keyStoreJksFile == null) { + keyStoreJksFile = File.createTempFile(KEY_STORE_PREFIX, + KeyStoreFileType.JKS.getDefaultFileExtension(), tempDir); + keyStoreJksFile.deleteOnExit(); + generateKeyStoreJksFile(); + } + return keyStoreJksFile; + } + + private void generateKeyStoreJksFile() throws IOException { + try (final FileOutputStream keyStoreOutputStream = new FileOutputStream(keyStoreJksFile)) { + byte[] bytes = X509TestHelpers.certAndPrivateKeyToJavaKeyStoreBytes(keyStoreCertificate, + keyStoreKeyPair.getPrivate(), keyStorePassword); + keyStoreOutputStream.write(bytes); + keyStoreOutputStream.flush(); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + } + + private File getKeyStorePemFile() throws IOException { + if (keyStorePemFile == null) { + try { + keyStorePemFile = File.createTempFile(KEY_STORE_PREFIX, + KeyStoreFileType.PEM.getDefaultFileExtension(), tempDir); + keyStorePemFile.deleteOnExit(); + generateKeyStorePemFile(); + } catch (OperatorCreationException e) { + throw new IOException(e); + } + } + return keyStorePemFile; + } + + private void generateKeyStorePemFile() throws IOException, OperatorCreationException { + FileUtils.writeStringToFile(keyStorePemFile, + X509TestHelpers.pemEncodeCertAndPrivateKey(keyStoreCertificate, keyStoreKeyPair.getPrivate(), + keyStorePassword), + StandardCharsets.US_ASCII, false); + } + + private File getKeyStorePkcs12File() throws IOException { + if (keyStorePkcs12File == null) { + keyStorePkcs12File = File.createTempFile(KEY_STORE_PREFIX, + KeyStoreFileType.PKCS12.getDefaultFileExtension(), tempDir); + keyStorePkcs12File.deleteOnExit(); + generateKeyStorePkcs12File(); + } + return keyStorePkcs12File; + } + + private void generateKeyStorePkcs12File() throws IOException { + try (final FileOutputStream keyStoreOutputStream = new FileOutputStream(keyStorePkcs12File)) { + byte[] bytes = X509TestHelpers.certAndPrivateKeyToPKCS12Bytes(keyStoreCertificate, + keyStoreKeyPair.getPrivate(), keyStorePassword); + keyStoreOutputStream.write(bytes); + keyStoreOutputStream.flush(); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + } + + private File getKeyStoreBcfksFile() throws IOException { + if (keyStoreBcfksFile == null) { + keyStoreBcfksFile = File.createTempFile(KEY_STORE_PREFIX, + KeyStoreFileType.BCFKS.getDefaultFileExtension(), tempDir); + keyStoreBcfksFile.deleteOnExit(); + generateKeyStoreBcfksFile(); + } + return keyStoreBcfksFile; + } + + private void generateKeyStoreBcfksFile() throws IOException { + try (final FileOutputStream keyStoreOutputStream = new FileOutputStream(keyStoreBcfksFile)) { + byte[] bytes = X509TestHelpers.certAndPrivateKeyToBCFKSBytes(keyStoreCertificate, + keyStoreKeyPair.getPrivate(), keyStorePassword); + keyStoreOutputStream.write(bytes); + keyStoreOutputStream.flush(); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + } + + /** + * Sets the SSL system properties such that the given X509Util object can be used to create SSL + * Contexts that will use the trust store and key store files created by this test context. + * Example usage: + * + *

+   *     X509TestContext testContext = ...; // create the test context
+   *     X509Util x509Util = new QuorumX509Util();
+   *     testContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
+   *     // The returned context will use the key store and trust store created by the test context.
+   *     SSLContext ctx = x509Util.getDefaultSSLContext();
+   * 
+ * + * @param keyStoreFileType the store file type to use for the key store (JKS, PEM, ...). + * @param trustStoreFileType the store file type to use for the trust store (JKS, PEM, ...). + * @throws IOException if there is an error creating the key store file or trust store file. + */ + public void setConfigurations(KeyStoreFileType keyStoreFileType, + KeyStoreFileType trustStoreFileType) throws IOException { + setKeystoreConfigurations(keyStoreFileType, conf); + conf.set(X509Util.TLS_CONFIG_TRUSTSTORE_LOCATION, + this.getTrustStoreFile(trustStoreFileType).getAbsolutePath()); + conf.set(X509Util.TLS_CONFIG_TRUSTSTORE_PASSWORD, String.valueOf(this.getTrustStorePassword())); + conf.set(X509Util.TLS_CONFIG_TRUSTSTORE_TYPE, trustStoreFileType.getPropertyValue()); + } + + /** + * Sets the KeyStore-related SSL system properties onto the given Configuration such that X509Util + * can be used to create SSL Contexts using that KeyStore. This can be used in special + * circumstances to inject a "bad" certificate where the keystore doesn't match the CA in the + * truststore. Or use it to create a connection without a truststore. + * @see #setConfigurations(KeyStoreFileType, KeyStoreFileType) which sets both keystore and + * truststore and is more applicable to general use. nnn + */ + public void setKeystoreConfigurations(KeyStoreFileType keyStoreFileType, Configuration confToSet) + throws IOException { + + confToSet.set(X509Util.TLS_CONFIG_KEYSTORE_LOCATION, + this.getKeyStoreFile(keyStoreFileType).getAbsolutePath()); + confToSet.set(X509Util.TLS_CONFIG_KEYSTORE_PASSWORD, + String.valueOf(this.getKeyStorePassword())); + confToSet.set(X509Util.TLS_CONFIG_KEYSTORE_TYPE, keyStoreFileType.getPropertyValue()); + } + + public void clearConfigurations() { + conf.unset(X509Util.TLS_CONFIG_KEYSTORE_LOCATION); + conf.unset(X509Util.TLS_CONFIG_KEYSTORE_PASSWORD); + conf.unset(X509Util.TLS_CONFIG_KEYSTORE_TYPE); + conf.unset(X509Util.TLS_CONFIG_TRUSTSTORE_LOCATION); + conf.unset(X509Util.TLS_CONFIG_TRUSTSTORE_PASSWORD); + conf.unset(X509Util.TLS_CONFIG_TRUSTSTORE_TYPE); + } + + /** + * Creates a clone of the current context, but injecting the passed certificate as the KeyStore + * cert. The new context's keystore path fields are nulled, so the next call to + * {@link #setConfigurations(KeyStoreFileType, KeyStoreFileType)}, + * {@link #setKeystoreConfigurations(KeyStoreFileType, Configuration)} , or + * {@link #getKeyStoreFile(KeyStoreFileType)} will create a new keystore with this certificate in + * place. + * @param cert the cert to replace + */ + public X509TestContext cloneWithNewKeystoreCert(X509Certificate cert) { + return new X509TestContext(tempDir, conf, trustStoreCertificate, trustStorePassword, + trustStoreKeyPair, trustStoreJksFile, trustStorePemFile, trustStorePkcs12File, + keyStoreKeyPair, keyStorePassword, cert); + } + + public void regenerateStores(X509KeyType keyStoreKeyType, X509KeyType trustStoreKeyType, + KeyStoreFileType keyStoreFileType, KeyStoreFileType trustStoreFileType) + throws GeneralSecurityException, IOException, OperatorCreationException { + + trustStoreKeyPair = X509TestHelpers.generateKeyPair(trustStoreKeyType); + keyStoreKeyPair = X509TestHelpers.generateKeyPair(keyStoreKeyType); + createCertificates(); + + switch (keyStoreFileType) { + case JKS: + generateKeyStoreJksFile(); + break; + case PEM: + generateKeyStorePemFile(); + break; + case BCFKS: + generateKeyStoreBcfksFile(); + break; + case PKCS12: + generateKeyStorePkcs12File(); + break; + } + + switch (trustStoreFileType) { + case JKS: + generateTrustStoreJksFile(); + break; + case PEM: + generateTrustStorePemFile(); + break; + case PKCS12: + generateTrustStorePkcs12File(); + break; + case BCFKS: + generateTrustStoreBcfksFile(); + break; + } + } + + private void createCertificates() + throws GeneralSecurityException, IOException, OperatorCreationException { + X500NameBuilder caNameBuilder = new X500NameBuilder(BCStyle.INSTANCE); + caNameBuilder.addRDN(BCStyle.CN, + MethodHandles.lookup().lookupClass().getCanonicalName() + " Root CA"); + trustStoreCertificate = + X509TestHelpers.newSelfSignedCACert(caNameBuilder.build(), trustStoreKeyPair); + + X500NameBuilder nameBuilder = new X500NameBuilder(BCStyle.INSTANCE); + nameBuilder.addRDN(BCStyle.CN, + MethodHandles.lookup().lookupClass().getCanonicalName() + " Zookeeper Test"); + keyStoreCertificate = newCert(nameBuilder.build()); + } + + /** + * Builder class, used for creating new instances of X509TestContext. + */ + public static class Builder { + + private final Configuration conf; + private File tempDir; + private X509KeyType trustStoreKeyType; + private char[] trustStorePassword; + private X509KeyType keyStoreKeyType; + private char[] keyStorePassword; + + /** + * Creates an empty builder with the given Configuration. + */ + public Builder(Configuration conf) { + this.conf = conf; + trustStoreKeyType = X509KeyType.EC; + keyStoreKeyType = X509KeyType.EC; + } + + /** + * Builds a new X509TestContext from this builder. + * @return a new X509TestContext + */ + public X509TestContext build() + throws IOException, GeneralSecurityException, OperatorCreationException { + KeyPair trustStoreKeyPair = X509TestHelpers.generateKeyPair(trustStoreKeyType); + KeyPair keyStoreKeyPair = X509TestHelpers.generateKeyPair(keyStoreKeyType); + return new X509TestContext(conf, tempDir, trustStoreKeyPair, trustStorePassword, + keyStoreKeyPair, keyStorePassword); + } + + /** + * Sets the temporary directory. Certificate and private key files will be created in this + * directory. + * @param tempDir the temp directory. + * @return this Builder. + */ + public Builder setTempDir(File tempDir) { + this.tempDir = tempDir; + return this; + } + + /** + * Sets the trust store key type. The CA key generated for the test context will be of this + * type. + * @param keyType the key type. + * @return this Builder. + */ + public Builder setTrustStoreKeyType(X509KeyType keyType) { + trustStoreKeyType = keyType; + return this; + } + + /** + * Sets the trust store password. Ignored for PEM trust stores, JKS trust stores will be + * encrypted with this password. + * @param password the password. + * @return this Builder. + */ + public Builder setTrustStorePassword(char[] password) { + trustStorePassword = password; + return this; + } + + /** + * Sets the key store key type. The private key generated for the test context will be of this + * type. + * @param keyType the key type. + * @return this Builder. + */ + public Builder setKeyStoreKeyType(X509KeyType keyType) { + keyStoreKeyType = keyType; + return this; + } + + /** + * Sets the key store password. The private key (PEM, JKS) and certificate (JKS only) will be + * encrypted with this password. + * @param password the password. + * @return this Builder. + */ + public Builder setKeyStorePassword(char[] password) { + keyStorePassword = password; + return this; + } + } + + /** + * Returns a new default-constructed Builder. + * @return a new Builder. + */ + public static Builder newBuilder(Configuration conf) { + return new Builder(conf); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestContextProvider.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestContextProvider.java new file mode 100644 index 000000000000..d65cdbe689dd --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestContextProvider.java @@ -0,0 +1,103 @@ +/* + * 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.io.crypto.tls; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import org.apache.hadoop.conf.Configuration; + +import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader; +import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache; + +/** + * Will cache X509TestContext to speed up tests. + */ +public class X509TestContextProvider { + + private static final class CacheKey { + private final X509KeyType caKeyType; + + private final X509KeyType certKeyType; + + private final char[] keyPassword; + + CacheKey(X509KeyType caKeyType, X509KeyType certKeyType, char[] keyPassword) { + this.caKeyType = caKeyType; + this.certKeyType = certKeyType; + this.keyPassword = keyPassword; + } + + @Override + public int hashCode() { + return Objects.hash(caKeyType, certKeyType, Arrays.hashCode(keyPassword)); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof CacheKey)) { + return false; + } + CacheKey other = (CacheKey) obj; + return caKeyType == other.caKeyType && certKeyType == other.certKeyType + && Arrays.equals(keyPassword, other.keyPassword); + } + } + + private final Configuration conf; + + private final File tempDir; + + private final LoadingCache ctxs = + CacheBuilder.newBuilder().build(new CacheLoader() { + + @Override + public X509TestContext load(CacheKey key) throws Exception { + return X509TestContext.newBuilder(conf).setTempDir(tempDir) + .setKeyStorePassword(key.keyPassword).setKeyStoreKeyType(key.certKeyType) + .setTrustStorePassword(key.keyPassword).setTrustStoreKeyType(key.caKeyType).build(); + } + }); + + public X509TestContextProvider(Configuration conf, File tempDir) { + this.conf = conf; + this.tempDir = tempDir; + } + + public X509TestContext get(X509KeyType caKeyType, X509KeyType certKeyType, char[] keyPassword) { + return ctxs.getUnchecked(new CacheKey(caKeyType, certKeyType, keyPassword)); + } + + static Collection defaultParams() { + List params = new ArrayList<>(); + int paramIndex = 0; + for (X509KeyType caKeyType : X509KeyType.values()) { + for (X509KeyType certKeyType : X509KeyType.values()) { + for (char[] keyPassword : new char[][] { "".toCharArray(), "pa$$w0rd".toCharArray() }) { + params.add(new Object[] { caKeyType, certKeyType, keyPassword, paramIndex++ }); + } + } + } + return params; + } + +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestHelpers.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestHelpers.java new file mode 100644 index 000000000000..56d3c8cb8599 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/tls/X509TestHelpers.java @@ -0,0 +1,472 @@ +/* + * 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.io.crypto.tls; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.StringWriter; +import java.math.BigInteger; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.security.GeneralSecurityException; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.KeyStore; +import java.security.PrivateKey; +import java.security.PublicKey; +import java.security.SecureRandom; +import java.security.cert.Certificate; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.security.spec.ECGenParameterSpec; +import java.security.spec.RSAKeyGenParameterSpec; +import java.time.LocalDate; +import java.time.ZoneId; +import org.apache.yetus.audience.InterfaceAudience; +import org.bouncycastle.asn1.DERIA5String; +import org.bouncycastle.asn1.DEROctetString; +import org.bouncycastle.asn1.pkcs.PKCSObjectIdentifiers; +import org.bouncycastle.asn1.x500.X500Name; +import org.bouncycastle.asn1.x509.AlgorithmIdentifier; +import org.bouncycastle.asn1.x509.BasicConstraints; +import org.bouncycastle.asn1.x509.ExtendedKeyUsage; +import org.bouncycastle.asn1.x509.Extension; +import org.bouncycastle.asn1.x509.GeneralName; +import org.bouncycastle.asn1.x509.GeneralNames; +import org.bouncycastle.asn1.x509.KeyPurposeId; +import org.bouncycastle.asn1.x509.KeyUsage; +import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo; +import org.bouncycastle.cert.X509CertificateHolder; +import org.bouncycastle.cert.X509v3CertificateBuilder; +import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter; +import org.bouncycastle.crypto.params.AsymmetricKeyParameter; +import org.bouncycastle.crypto.util.PrivateKeyFactory; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.openssl.jcajce.JcaPEMWriter; +import org.bouncycastle.openssl.jcajce.JcaPKCS8Generator; +import org.bouncycastle.openssl.jcajce.JceOpenSSLPKCS8EncryptorBuilder; +import org.bouncycastle.operator.ContentSigner; +import org.bouncycastle.operator.DefaultDigestAlgorithmIdentifierFinder; +import org.bouncycastle.operator.DefaultSignatureAlgorithmIdentifierFinder; +import org.bouncycastle.operator.OperatorCreationException; +import org.bouncycastle.operator.OutputEncryptor; +import org.bouncycastle.operator.bc.BcContentSignerBuilder; +import org.bouncycastle.operator.bc.BcECContentSignerBuilder; +import org.bouncycastle.operator.bc.BcRSAContentSignerBuilder; + +/** + * This class contains helper methods for creating X509 certificates and key pairs, and for + * serializing them to JKS, PEM or other keystore type files. + *

+ * This file has been copied from the Apache ZooKeeper project. + * @see Base + * revision + */ +@InterfaceAudience.Private +final class X509TestHelpers { + + private static final SecureRandom PRNG = new SecureRandom(); + private static final int DEFAULT_RSA_KEY_SIZE_BITS = 2048; + private static final BigInteger DEFAULT_RSA_PUB_EXPONENT = RSAKeyGenParameterSpec.F4; // 65537 + private static final String DEFAULT_ELLIPTIC_CURVE_NAME = "secp256r1"; + // Per RFC 5280 section 4.1.2.2, X509 certificates can use up to 20 bytes == 160 bits for serial + // numbers. + private static final int SERIAL_NUMBER_MAX_BITS = 20 * Byte.SIZE; + + /** + * Uses the private key of the given key pair to create a self-signed CA certificate with the + * public half of the key pair and the given subject and expiration. The issuer of the new cert + * will be equal to the subject. Returns the new certificate. The returned certificate should be + * used as the trust store. The private key of the input key pair should be used to sign + * certificates that are used by test peers to establish TLS connections to each other. + * @param subject the subject of the new certificate being created. + * @param keyPair the key pair to use. The public key will be embedded in the new certificate, and + * the private key will be used to self-sign the certificate. + * @return a new self-signed CA certificate. + */ + public static X509Certificate newSelfSignedCACert(X500Name subject, KeyPair keyPair) + throws IOException, OperatorCreationException, GeneralSecurityException { + LocalDate now = LocalDate.now(ZoneId.systemDefault()); + X509v3CertificateBuilder builder = initCertBuilder(subject, // for self-signed certs, + // issuer == subject + now, now.plusDays(1), subject, keyPair.getPublic()); + builder.addExtension(Extension.basicConstraints, true, new BasicConstraints(true)); // is a CA + builder.addExtension(Extension.keyUsage, true, + new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyCertSign | KeyUsage.cRLSign)); + return buildAndSignCertificate(keyPair.getPrivate(), builder); + } + + /** + * Using the private key of the given CA key pair and the Subject of the given CA cert as the + * Issuer, issues a new cert with the given subject and public key. The returned certificate, + * combined with the private key half of the certPublicKey, should be used as the key + * store. + * @param caCert the certificate of the CA that's doing the signing. + * @param caKeyPair the key pair of the CA. The private key will be used to sign. The public + * key must match the public key in the caCert. + * @param certSubject the subject field of the new cert being issued. + * @param certPublicKey the public key of the new cert being issued. + * @return a new certificate signed by the CA's private key. + */ + public static X509Certificate newCert(X509Certificate caCert, KeyPair caKeyPair, + X500Name certSubject, PublicKey certPublicKey) + throws IOException, OperatorCreationException, GeneralSecurityException { + return newCert(caCert, caKeyPair, certSubject, certPublicKey, getLocalhostSubjectAltNames()); + } + + /** + * Using the private key of the given CA key pair and the Subject of the given CA cert as the + * Issuer, issues a new cert with the given subject and public key. The returned certificate, + * combined with the private key half of the certPublicKey, should be used as the key + * store. + * @param caCert the certificate of the CA that's doing the signing. + * @param caKeyPair the key pair of the CA. The private key will be used to sign. The public + * key must match the public key in the caCert. + * @param certSubject the subject field of the new cert being issued. + * @param certPublicKey the public key of the new cert being issued. + * @param subjectAltNames the subject alternative names to use, or null if none + * @return a new certificate signed by the CA's private key. + */ + public static X509Certificate newCert(X509Certificate caCert, KeyPair caKeyPair, + X500Name certSubject, PublicKey certPublicKey, GeneralNames subjectAltNames) + throws IOException, OperatorCreationException, GeneralSecurityException { + if (!caKeyPair.getPublic().equals(caCert.getPublicKey())) { + throw new IllegalArgumentException( + "CA private key does not match the public key in " + "the CA cert"); + } + LocalDate now = LocalDate.now(ZoneId.systemDefault()); + X509v3CertificateBuilder builder = initCertBuilder(new X500Name(caCert.getIssuerDN().getName()), + now, now.plusDays(1), certSubject, certPublicKey); + builder.addExtension(Extension.basicConstraints, true, new BasicConstraints(false)); // not a CA + builder.addExtension(Extension.keyUsage, true, + new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyEncipherment)); + builder.addExtension(Extension.extendedKeyUsage, true, new ExtendedKeyUsage( + new KeyPurposeId[] { KeyPurposeId.id_kp_serverAuth, KeyPurposeId.id_kp_clientAuth })); + + if (subjectAltNames != null) { + builder.addExtension(Extension.subjectAlternativeName, false, subjectAltNames); + } + return buildAndSignCertificate(caKeyPair.getPrivate(), builder); + } + + /** + * Returns subject alternative names for "localhost". + * @return the subject alternative names for "localhost". + */ + private static GeneralNames getLocalhostSubjectAltNames() throws UnknownHostException { + InetAddress[] localAddresses = InetAddress.getAllByName("localhost"); + GeneralName[] generalNames = new GeneralName[localAddresses.length + 1]; + for (int i = 0; i < localAddresses.length; i++) { + generalNames[i] = + new GeneralName(GeneralName.iPAddress, new DEROctetString(localAddresses[i].getAddress())); + } + generalNames[generalNames.length - 1] = + new GeneralName(GeneralName.dNSName, new DERIA5String("localhost")); + return new GeneralNames(generalNames); + } + + /** + * Helper method for newSelfSignedCACert() and newCert(). Initializes a X509v3CertificateBuilder + * with logic that's common to both methods. + * @param issuer Issuer field of the new cert. + * @param notBefore date before which the new cert is not valid. + * @param notAfter date after which the new cert is not valid. + * @param subject Subject field of the new cert. + * @param subjectPublicKey public key to store in the new cert. + * @return a X509v3CertificateBuilder that can be further customized to finish creating the new + * cert. + */ + private static X509v3CertificateBuilder initCertBuilder(X500Name issuer, LocalDate notBefore, + LocalDate notAfter, X500Name subject, PublicKey subjectPublicKey) { + return new X509v3CertificateBuilder(issuer, new BigInteger(SERIAL_NUMBER_MAX_BITS, PRNG), + java.sql.Date.valueOf(notBefore), java.sql.Date.valueOf(notAfter), subject, + SubjectPublicKeyInfo.getInstance(subjectPublicKey.getEncoded())); + } + + /** + * Signs the certificate being built by the given builder using the given private key and returns + * the certificate. + * @param privateKey the private key to sign the certificate with. + * @param builder the cert builder that contains the certificate data. + * @return the signed certificate. + */ + private static X509Certificate buildAndSignCertificate(PrivateKey privateKey, + X509v3CertificateBuilder builder) + throws IOException, OperatorCreationException, CertificateException { + BcContentSignerBuilder signerBuilder; + if (privateKey.getAlgorithm().contains("RSA")) { // a little hacky way to detect key type, but + // it works + AlgorithmIdentifier signatureAlgorithm = + new DefaultSignatureAlgorithmIdentifierFinder().find("SHA256WithRSAEncryption"); + AlgorithmIdentifier digestAlgorithm = + new DefaultDigestAlgorithmIdentifierFinder().find(signatureAlgorithm); + signerBuilder = new BcRSAContentSignerBuilder(signatureAlgorithm, digestAlgorithm); + } else { // if not RSA, assume EC + AlgorithmIdentifier signatureAlgorithm = + new DefaultSignatureAlgorithmIdentifierFinder().find("SHA256withECDSA"); + AlgorithmIdentifier digestAlgorithm = + new DefaultDigestAlgorithmIdentifierFinder().find(signatureAlgorithm); + signerBuilder = new BcECContentSignerBuilder(signatureAlgorithm, digestAlgorithm); + } + AsymmetricKeyParameter privateKeyParam = PrivateKeyFactory.createKey(privateKey.getEncoded()); + ContentSigner signer = signerBuilder.build(privateKeyParam); + return toX509Cert(builder.build(signer)); + } + + /** + * Generates a new asymmetric key pair of the given type. + * @param keyType the type of key pair to generate. + * @return the new key pair. + * @throws GeneralSecurityException if your java crypto providers are messed up. + */ + public static KeyPair generateKeyPair(X509KeyType keyType) throws GeneralSecurityException { + switch (keyType) { + case RSA: + return generateRSAKeyPair(); + case EC: + return generateECKeyPair(); + default: + throw new IllegalArgumentException("Invalid X509KeyType"); + } + } + + /** + * Generates an RSA key pair with a 2048-bit private key and F4 (65537) as the public exponent. + * @return the key pair. + */ + public static KeyPair generateRSAKeyPair() throws GeneralSecurityException { + KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA"); + RSAKeyGenParameterSpec keyGenSpec = + new RSAKeyGenParameterSpec(DEFAULT_RSA_KEY_SIZE_BITS, DEFAULT_RSA_PUB_EXPONENT); + keyGen.initialize(keyGenSpec, PRNG); + return keyGen.generateKeyPair(); + } + + /** + * Generates an elliptic curve key pair using the "secp256r1" aka "prime256v1" aka "NIST P-256" + * curve. + * @return the key pair. + */ + public static KeyPair generateECKeyPair() throws GeneralSecurityException { + KeyPairGenerator keyGen = KeyPairGenerator.getInstance("EC"); + keyGen.initialize(new ECGenParameterSpec(DEFAULT_ELLIPTIC_CURVE_NAME), PRNG); + return keyGen.generateKeyPair(); + } + + /** + * PEM-encodes the given X509 certificate and private key (compatible with OpenSSL), optionally + * protecting the private key with a password. Concatenates them both and returns the result as a + * single string. This creates the PEM encoding of a key store. + * @param cert the X509 certificate to PEM-encode. + * @param privateKey the private key to PEM-encode. + * @param keyPassword an optional key password. If empty or null, the private key will not be + * encrypted. + * @return a String containing the PEM encodings of the certificate and private key. + * @throws IOException if converting the certificate or private key to PEM format + * fails. + * @throws OperatorCreationException if constructing the encryptor from the given password fails. + */ + public static String pemEncodeCertAndPrivateKey(X509Certificate cert, PrivateKey privateKey, + char[] keyPassword) throws IOException, OperatorCreationException { + return pemEncodeX509Certificate(cert) + "\n" + pemEncodePrivateKey(privateKey, keyPassword); + } + + /** + * PEM-encodes the given private key (compatible with OpenSSL), optionally protecting it with a + * password, and returns the result as a String. + * @param key the private key. + * @param password an optional key password. If empty or null, the private key will not be + * encrypted. + * @return a String containing the PEM encoding of the private key. + * @throws IOException if converting the key to PEM format fails. + * @throws OperatorCreationException if constructing the encryptor from the given password fails. + */ + public static String pemEncodePrivateKey(PrivateKey key, char[] password) + throws IOException, OperatorCreationException { + StringWriter stringWriter = new StringWriter(); + JcaPEMWriter pemWriter = new JcaPEMWriter(stringWriter); + OutputEncryptor encryptor = null; + if (password != null && password.length > 0) { + encryptor = + new JceOpenSSLPKCS8EncryptorBuilder(PKCSObjectIdentifiers.pbeWithSHAAnd3_KeyTripleDES_CBC) + .setProvider(BouncyCastleProvider.PROVIDER_NAME).setRandom(PRNG).setPasssword(password) + .build(); + } + pemWriter.writeObject(new JcaPKCS8Generator(key, encryptor)); + pemWriter.close(); + return stringWriter.toString(); + } + + /** + * PEM-encodes the given X509 certificate (compatible with OpenSSL) and returns the result as a + * String. + * @param cert the certificate. + * @return a String containing the PEM encoding of the certificate. + * @throws IOException if converting the certificate to PEM format fails. + */ + public static String pemEncodeX509Certificate(X509Certificate cert) throws IOException { + StringWriter stringWriter = new StringWriter(); + JcaPEMWriter pemWriter = new JcaPEMWriter(stringWriter); + pemWriter.writeObject(cert); + pemWriter.close(); + return stringWriter.toString(); + } + + /** + * Encodes the given X509Certificate as a JKS TrustStore, optionally protecting the cert with a + * password (though it's unclear why one would do this since certificates only contain public + * information and do not need to be kept secret). Returns the byte array encoding of the trust + * store, which may be written to a file and loaded to instantiate the trust store at a later + * point or in another process. + * @param cert the certificate to serialize. + * @param keyPassword an optional password to encrypt the trust store. If empty or null, the cert + * will not be encrypted. + * @return the serialized bytes of the JKS trust store. + */ + public static byte[] certToJavaTrustStoreBytes(X509Certificate cert, char[] keyPassword) + throws IOException, GeneralSecurityException { + KeyStore trustStore = KeyStore.getInstance(KeyStore.getDefaultType()); + return certToTrustStoreBytes(cert, keyPassword, trustStore); + } + + /** + * Encodes the given X509Certificate as a PKCS12 TrustStore, optionally protecting the cert with a + * password (though it's unclear why one would do this since certificates only contain public + * information and do not need to be kept secret). Returns the byte array encoding of the trust + * store, which may be written to a file and loaded to instantiate the trust store at a later + * point or in another process. + * @param cert the certificate to serialize. + * @param keyPassword an optional password to encrypt the trust store. If empty or null, the cert + * will not be encrypted. + * @return the serialized bytes of the PKCS12 trust store. + */ + public static byte[] certToPKCS12TrustStoreBytes(X509Certificate cert, char[] keyPassword) + throws IOException, GeneralSecurityException { + KeyStore trustStore = KeyStore.getInstance("PKCS12"); + return certToTrustStoreBytes(cert, keyPassword, trustStore); + } + + /** + * Encodes the given X509Certificate as a BCFKS TrustStore, optionally protecting the cert with a + * password (though it's unclear why one would do this since certificates only contain public + * information and do not need to be kept secret). Returns the byte array encoding of the trust + * store, which may be written to a file and loaded to instantiate the trust store at a later + * point or in another process. + * @param cert the certificate to serialize. + * @param keyPassword an optional password to encrypt the trust store. If empty or null, the cert + * will not be encrypted. + * @return the serialized bytes of the BCFKS trust store. nn + */ + public static byte[] certToBCFKSTrustStoreBytes(X509Certificate cert, char[] keyPassword) + throws IOException, GeneralSecurityException { + KeyStore trustStore = KeyStore.getInstance("BCFKS"); + return certToTrustStoreBytes(cert, keyPassword, trustStore); + } + + private static byte[] certToTrustStoreBytes(X509Certificate cert, char[] keyPassword, + KeyStore trustStore) throws IOException, GeneralSecurityException { + trustStore.load(null, keyPassword); + trustStore.setCertificateEntry(cert.getSubjectDN().toString(), cert); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + trustStore.store(outputStream, keyPassword); + outputStream.flush(); + byte[] result = outputStream.toByteArray(); + outputStream.close(); + return result; + } + + /** + * Encodes the given X509Certificate and private key as a JKS KeyStore, optionally protecting the + * private key (and possibly the cert?) with a password. Returns the byte array encoding of the + * key store, which may be written to a file and loaded to instantiate the key store at a later + * point or in another process. + * @param cert the X509 certificate to serialize. + * @param privateKey the private key to serialize. + * @param keyPassword an optional key password. If empty or null, the private key will not be + * encrypted. + * @return the serialized bytes of the JKS key store. + */ + public static byte[] certAndPrivateKeyToJavaKeyStoreBytes(X509Certificate cert, + PrivateKey privateKey, char[] keyPassword) throws IOException, GeneralSecurityException { + KeyStore keyStore = KeyStore.getInstance(KeyStore.getDefaultType()); + return certAndPrivateKeyToBytes(cert, privateKey, keyPassword, keyStore); + } + + /** + * Encodes the given X509Certificate and private key as a PKCS12 KeyStore, optionally protecting + * the private key (and possibly the cert?) with a password. Returns the byte array encoding of + * the key store, which may be written to a file and loaded to instantiate the key store at a + * later point or in another process. + * @param cert the X509 certificate to serialize. + * @param privateKey the private key to serialize. + * @param keyPassword an optional key password. If empty or null, the private key will not be + * encrypted. + * @return the serialized bytes of the PKCS12 key store. + */ + public static byte[] certAndPrivateKeyToPKCS12Bytes(X509Certificate cert, PrivateKey privateKey, + char[] keyPassword) throws IOException, GeneralSecurityException { + KeyStore keyStore = KeyStore.getInstance("PKCS12"); + return certAndPrivateKeyToBytes(cert, privateKey, keyPassword, keyStore); + } + + /** + * Encodes the given X509Certificate and private key as a BCFKS KeyStore, optionally protecting + * the private key (and possibly the cert?) with a password. Returns the byte array encoding of + * the key store, which may be written to a file and loaded to instantiate the key store at a + * later point or in another process. + * @param cert the X509 certificate to serialize. + * @param privateKey the private key to serialize. + * @param keyPassword an optional key password. If empty or null, the private key will not be + * encrypted. + * @return the serialized bytes of the BCFKS key store. nn + */ + public static byte[] certAndPrivateKeyToBCFKSBytes(X509Certificate cert, PrivateKey privateKey, + char[] keyPassword) throws IOException, GeneralSecurityException { + KeyStore keyStore = KeyStore.getInstance("BCFKS"); + return certAndPrivateKeyToBytes(cert, privateKey, keyPassword, keyStore); + } + + private static byte[] certAndPrivateKeyToBytes(X509Certificate cert, PrivateKey privateKey, + char[] keyPassword, KeyStore keyStore) throws IOException, GeneralSecurityException { + keyStore.load(null, keyPassword); + keyStore.setKeyEntry("key", privateKey, keyPassword, new Certificate[] { cert }); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + keyStore.store(outputStream, keyPassword); + outputStream.flush(); + byte[] result = outputStream.toByteArray(); + outputStream.close(); + return result; + } + + /** + * Convenience method to convert a bouncycastle X509CertificateHolder to a java X509Certificate. + * @param certHolder a bouncycastle X509CertificateHolder. + * @return a java X509Certificate + * @throws CertificateException if the conversion fails. + */ + public static X509Certificate toX509Cert(X509CertificateHolder certHolder) + throws CertificateException { + return new JcaX509CertificateConverter().setProvider(BouncyCastleProvider.PROVIDER_NAME) + .getCertificate(certHolder); + } + + private X509TestHelpers() { + // empty + } +} diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java index ebd9057ac241..b021de2d73e7 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java @@ -69,7 +69,7 @@ public IntegrationTestRpcClient() { protected AbstractRpcClient createRpcClient(Configuration conf, boolean isSyncClient) { return isSyncClient ? new BlockingRpcClient(conf) : new NettyRpcClient(conf) { @Override - Codec getCodec() { + protected Codec getCodec() { return null; } }; diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index 7a459fed0d50..c09b5feeb80e 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -346,6 +346,16 @@ log4j-1.2-api test + + org.bouncycastle + bcprov-jdk15on + test + + + org.bouncycastle + bcpkix-jdk15on + test + org.skyscreamer jsonassert diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyHBaseSaslRpcServerHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyHBaseSaslRpcServerHandler.java new file mode 100644 index 000000000000..387318888a00 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyHBaseSaslRpcServerHandler.java @@ -0,0 +1,115 @@ +/* + * 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 java.io.IOException; +import org.apache.hadoop.hbase.security.HBaseSaslRpcServer; +import org.apache.hadoop.hbase.security.SaslStatus; +import org.apache.hadoop.hbase.security.SaslUnwrapHandler; +import org.apache.hadoop.hbase.security.SaslWrapHandler; +import org.apache.hadoop.hbase.util.NettyFutureUtils; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; +import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline; +import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; +import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder; + +/** + * Implement SASL negotiation logic for rpc server. + */ +class NettyHBaseSaslRpcServerHandler extends SimpleChannelInboundHandler { + + private static final Logger LOG = LoggerFactory.getLogger(NettyHBaseSaslRpcServerHandler.class); + + static final String DECODER_NAME = "SaslNegotiationDecoder"; + + private final NettyRpcServer rpcServer; + + private final NettyServerRpcConnection conn; + + NettyHBaseSaslRpcServerHandler(NettyRpcServer rpcServer, NettyServerRpcConnection conn) { + this.rpcServer = rpcServer; + this.conn = conn; + } + + private void doResponse(ChannelHandlerContext ctx, SaslStatus status, Writable rv, + String errorClass, String error) throws IOException { + // In my testing, have noticed that sasl messages are usually + // in the ballpark of 100-200. That's why the initial capacity is 256. + ByteBuf resp = ctx.alloc().buffer(256); + try (ByteBufOutputStream out = new ByteBufOutputStream(resp)) { + out.writeInt(status.state); // write status + if (status == SaslStatus.SUCCESS) { + rv.write(out); + } else { + WritableUtils.writeString(out, errorClass); + WritableUtils.writeString(out, error); + } + } + NettyFutureUtils.safeWriteAndFlush(ctx, resp); + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { + LOG.debug("Read input token of size={} for processing by saslServer.evaluateResponse()", + msg.readableBytes()); + HBaseSaslRpcServer saslServer = conn.getOrCreateSaslServer(); + byte[] saslToken = new byte[msg.readableBytes()]; + msg.readBytes(saslToken, 0, saslToken.length); + byte[] replyToken = saslServer.evaluateResponse(saslToken); + if (replyToken != null) { + LOG.debug("Will send token of size {} from saslServer.", replyToken.length); + doResponse(ctx, SaslStatus.SUCCESS, new BytesWritable(replyToken), null, null); + } + if (saslServer.isComplete()) { + conn.finishSaslNegotiation(); + String qop = saslServer.getNegotiatedQop(); + boolean useWrap = qop != null && !"auth".equalsIgnoreCase(qop); + ChannelPipeline p = ctx.pipeline(); + if (useWrap) { + p.addBefore(DECODER_NAME, null, new SaslWrapHandler(saslServer::wrap)).addLast( + new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4), + new SaslUnwrapHandler(saslServer::unwrap)); + } + conn.setupHandler(); + p.remove(this); + p.remove(DECODER_NAME); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + LOG.error("Error when doing SASL handshade, provider={}", conn.provider, cause); + Throwable sendToClient = HBaseSaslRpcServer.unwrap(cause); + doResponse(ctx, SaslStatus.ERROR, null, sendToClient.getClass().getName(), + sendToClient.getLocalizedMessage()); + rpcServer.metrics.authenticationFailure(); + String clientIP = this.toString(); + // attempting user could be null + RpcServer.AUDITLOG.warn("{}{}: {}", RpcServer.AUTH_FAILED_FOR, clientIP, + conn.saslServer != null ? conn.saslServer.getAttemptingUser() : "Unknown"); + NettyFutureUtils.safeClose(ctx); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java index dec00424e73d..b60cad10189c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java @@ -38,22 +38,19 @@ * @since 2.0.0 */ @InterfaceAudience.Private -public class NettyRpcFrameDecoder extends ByteToMessageDecoder { +class NettyRpcFrameDecoder extends ByteToMessageDecoder { private static int FRAME_LENGTH_FIELD_LENGTH = 4; private final int maxFrameLength; + final NettyServerRpcConnection connection; + private boolean requestTooBig; private boolean requestTooBigSent; private String requestTooBigMessage; - public NettyRpcFrameDecoder(int maxFrameLength) { + public NettyRpcFrameDecoder(int maxFrameLength, NettyServerRpcConnection connection) { this.maxFrameLength = maxFrameLength; - } - - NettyServerRpcConnection connection; - - void setConnection(NettyServerRpcConnection connection) { this.connection = connection; } @@ -80,10 +77,10 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) t if (frameLength > maxFrameLength) { requestTooBig = true; - requestTooBigMessage = "RPC data length of " + frameLength + " received from " - + connection.getHostAddress() + " is greater than max allowed " - + connection.rpcServer.maxRequestSize + ". Set \"" + SimpleRpcServer.MAX_REQUEST_SIZE - + "\" on server to override this limit (not recommended)"; + requestTooBigMessage = + "RPC data length of " + frameLength + " received from " + connection.getHostAddress() + + " is greater than max allowed " + connection.rpcServer.maxRequestSize + ". Set \"" + + RpcServer.MAX_REQUEST_SIZE + "\" on server to override this limit (not recommended)"; NettyRpcServer.LOG.warn(requestTooBigMessage); @@ -137,7 +134,7 @@ private void handleTooBigRequest(ChannelHandlerContext ctx, ByteBuf in) throws I // Make sure the client recognizes the underlying exception // Otherwise, throw a DoNotRetryIOException. if ( - VersionInfoUtil.hasMinimumVersion(connection.connectionHeader.getVersionInfo(), + VersionInfoUtil.hasMinimumVersion(connection.getVersionInfo(), RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION) ) { reqTooBig.setResponse(null, null, reqTooBigEx, requestTooBigMessage); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java index 1c266d32f82d..f3ead471fe61 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java @@ -17,15 +17,24 @@ */ package org.apache.hadoop.hbase.ipc; +import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.HBASE_SERVER_NETTY_TLS_ENABLED; +import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT; +import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.TLS_CONFIG_REVERSE_DNS_LOOKUP_ENABLED; + import java.io.IOException; import java.io.InterruptedIOException; import java.net.InetSocketAddress; +import java.net.SocketAddress; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.exceptions.X509Exception; +import org.apache.hadoop.hbase.io.FileChangeWatcher; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.security.HBasePolicyProvider; @@ -54,6 +63,9 @@ import org.apache.hbase.thirdparty.io.netty.channel.group.ChannelGroup; import org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroup; import org.apache.hbase.thirdparty.io.netty.handler.codec.FixedLengthFrameDecoder; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.OptionalSslHandler; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler; import org.apache.hbase.thirdparty.io.netty.util.concurrent.GlobalEventExecutor; /** @@ -85,6 +97,9 @@ public class NettyRpcServer extends RpcServer { private final Channel serverChannel; final ChannelGroup allChannels = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE, true); private final ByteBufAllocator channelAllocator; + private final AtomicReference sslContextForServer = new AtomicReference<>(); + private final AtomicReference keyStoreWatcher = new AtomicReference<>(); + private final AtomicReference trustStoreWatcher = new AtomicReference<>(); public NettyRpcServer(Server server, String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler, @@ -113,11 +128,11 @@ protected void initChannel(Channel ch) throws Exception { ChannelPipeline pipeline = ch.pipeline(); FixedLengthFrameDecoder preambleDecoder = new FixedLengthFrameDecoder(6); preambleDecoder.setSingleDecode(true); - pipeline.addLast("preambleDecoder", preambleDecoder); - pipeline.addLast("preambleHandler", createNettyRpcServerPreambleHandler()); - pipeline.addLast("frameDecoder", new NettyRpcFrameDecoder(maxRequestSize)); - pipeline.addLast("decoder", new NettyRpcServerRequestDecoder(allChannels, metrics)); - pipeline.addLast("encoder", new NettyRpcServerResponseEncoder(metrics)); + if (conf.getBoolean(HBASE_SERVER_NETTY_TLS_ENABLED, false)) { + initSSL(pipeline, conf.getBoolean(HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, true)); + } + pipeline.addLast(NettyRpcServerPreambleHandler.DECODER_NAME, preambleDecoder) + .addLast(createNettyRpcServerPreambleHandler()); } }); try { @@ -160,6 +175,7 @@ private ByteBufAllocator getChannelAllocator(Configuration conf) throws IOExcept } } + // will be overriden in tests @InterfaceAudience.Private protected NettyRpcServerPreambleHandler createNettyRpcServerPreambleHandler() { return new NettyRpcServerPreambleHandler(NettyRpcServer.this); @@ -191,6 +207,14 @@ public synchronized void stop() { return; } LOG.info("Stopping server on " + this.serverChannel.localAddress()); + FileChangeWatcher ks = keyStoreWatcher.getAndSet(null); + if (ks != null) { + ks.stop(); + } + FileChangeWatcher ts = trustStoreWatcher.getAndSet(null); + if (ts != null) { + ts.stop(); + } if (authTokenSecretMgr != null) { authTokenSecretMgr.stop(); authTokenSecretMgr = null; @@ -237,4 +261,59 @@ public Pair call(BlockingService service, MethodDescriptor -1, null, receiveTime, timeout, bbAllocator, cellBlockBuilder, null); return call(fakeCall, status); } + + private void initSSL(ChannelPipeline p, boolean supportPlaintext) + throws X509Exception, IOException { + SslContext nettySslContext = getSslContext(); + + if (supportPlaintext) { + p.addLast("ssl", new OptionalSslHandler(nettySslContext)); + LOG.debug("Dual mode SSL handler added for channel: {}", p.channel()); + } else { + SocketAddress remoteAddress = p.channel().remoteAddress(); + SslHandler sslHandler; + + if (remoteAddress instanceof InetSocketAddress) { + InetSocketAddress remoteInetAddress = (InetSocketAddress) remoteAddress; + String host; + + if (conf.getBoolean(TLS_CONFIG_REVERSE_DNS_LOOKUP_ENABLED, true)) { + host = remoteInetAddress.getHostName(); + } else { + host = remoteInetAddress.getHostString(); + } + + int port = remoteInetAddress.getPort(); + + /* + * our HostnameVerifier gets the host name from SSLEngine, so we have to construct the + * engine properly by passing the remote address + */ + sslHandler = nettySslContext.newHandler(p.channel().alloc(), host, port); + } else { + sslHandler = nettySslContext.newHandler(p.channel().alloc()); + } + + p.addLast("ssl", sslHandler); + LOG.debug("SSL handler added for channel: {}", p.channel()); + } + } + + SslContext getSslContext() throws X509Exception, IOException { + SslContext result = sslContextForServer.get(); + if (result == null) { + result = X509Util.createSslContextForServer(conf); + if (!sslContextForServer.compareAndSet(null, result)) { + // lost the race, another thread already set the value + result = sslContextForServer.get(); + } else if ( + keyStoreWatcher.get() == null && trustStoreWatcher.get() == null + && conf.getBoolean(X509Util.TLS_CERT_RELOAD, false) + ) { + X509Util.enableCertFileReloading(conf, keyStoreWatcher, trustStoreWatcher, + () -> sslContextForServer.set(null)); + } + } + return result; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java index cf2551e1c087..ca25dea17fe2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.ipc; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.util.NettyFutureUtils; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; @@ -25,6 +26,7 @@ import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline; import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; +import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder; /** * Handle connection preamble. @@ -33,6 +35,8 @@ @InterfaceAudience.Private class NettyRpcServerPreambleHandler extends SimpleChannelInboundHandler { + static final String DECODER_NAME = "preambleDecoder"; + private final NettyRpcServer rpcServer; public NettyRpcServerPreambleHandler(NettyRpcServer rpcServer) { @@ -50,12 +54,29 @@ protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Excep return; } ChannelPipeline p = ctx.pipeline(); - ((NettyRpcFrameDecoder) p.get("frameDecoder")).setConnection(conn); - ((NettyRpcServerRequestDecoder) p.get("decoder")).setConnection(conn); + if (conn.useSasl) { + LengthFieldBasedFrameDecoder decoder = + new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4); + decoder.setSingleDecode(true); + p.addLast(NettyHBaseSaslRpcServerHandler.DECODER_NAME, decoder); + p.addLast(new NettyHBaseSaslRpcServerHandler(rpcServer, conn)); + } else { + conn.setupHandler(); + } + // add first and then remove, so the single decode decoder will pass the remaining bytes to the + // handler above. p.remove(this); - p.remove("preambleDecoder"); + p.remove(DECODER_NAME); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + NettyRpcServer.LOG.warn("Connection {}; caught unexpected downstream exception.", + ctx.channel().remoteAddress(), cause); + NettyFutureUtils.safeClose(ctx); } + // will be overridden in tests protected NettyServerRpcConnection createNettyServerRpcConnection(Channel channel) { return new NettyServerRpcConnection(rpcServer, channel); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java index cc8b07702b45..2e489e9ab05e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java @@ -17,64 +17,42 @@ */ package org.apache.hadoop.hbase.ipc; +import org.apache.hadoop.hbase.util.NettyFutureUtils; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; -import org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter; -import org.apache.hbase.thirdparty.io.netty.channel.group.ChannelGroup; +import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; /** * Decoder for rpc request. * @since 2.0.0 */ @InterfaceAudience.Private -class NettyRpcServerRequestDecoder extends ChannelInboundHandlerAdapter { - - private final ChannelGroup allChannels; +class NettyRpcServerRequestDecoder extends SimpleChannelInboundHandler { private final MetricsHBaseServer metrics; - public NettyRpcServerRequestDecoder(ChannelGroup allChannels, MetricsHBaseServer metrics) { - this.allChannels = allChannels; - this.metrics = metrics; - } - - private NettyServerRpcConnection connection; + private final NettyServerRpcConnection connection; - void setConnection(NettyServerRpcConnection connection) { + public NettyRpcServerRequestDecoder(MetricsHBaseServer metrics, + NettyServerRpcConnection connection) { + super(false); + this.metrics = metrics; this.connection = connection; } @Override - public void channelActive(ChannelHandlerContext ctx) throws Exception { - allChannels.add(ctx.channel()); - NettyRpcServer.LOG.trace("Connection {}; # active connections={}", - ctx.channel().remoteAddress(), (allChannels.size() - 1)); - super.channelActive(ctx); + public void exceptionCaught(ChannelHandlerContext ctx, Throwable e) { + NettyRpcServer.LOG.warn("Connection {}; caught unexpected downstream exception.", + ctx.channel().remoteAddress(), e); + NettyFutureUtils.safeClose(ctx); } @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - ByteBuf input = (ByteBuf) msg; + protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { // 4 bytes length field - metrics.receivedBytes(input.readableBytes() + 4); - connection.process(input); - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - allChannels.remove(ctx.channel()); - NettyRpcServer.LOG.trace("Disconnection {}; # active connections={}", - ctx.channel().remoteAddress(), (allChannels.size() - 1)); - super.channelInactive(ctx); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable e) { - allChannels.remove(ctx.channel()); - NettyRpcServer.LOG.trace("Connection {}; caught unexpected downstream exception.", - ctx.channel().remoteAddress(), e); - ctx.channel().close(); + metrics.receivedBytes(msg.readableBytes() + 4); + connection.process(msg); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java index 58be13769532..54c105802c55 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java @@ -20,12 +20,12 @@ import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.NettyFutureUtils; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; @@ -33,7 +33,6 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Message; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.channel.Channel; -import org.apache.hbase.thirdparty.io.netty.util.ReferenceCountUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; @@ -49,10 +48,16 @@ class NettyServerRpcConnection extends ServerRpcConnection { NettyServerRpcConnection(NettyRpcServer rpcServer, Channel channel) { super(rpcServer); this.channel = channel; + rpcServer.allChannels.add(channel); + NettyRpcServer.LOG.trace("Connection {}; # active connections={}", channel.remoteAddress(), + rpcServer.allChannels.size() - 1); // register close hook to release resources - channel.closeFuture().addListener(f -> { + NettyFutureUtils.addListener(channel.closeFuture(), f -> { disposeSasl(); callCleanupIfNeeded(); + NettyRpcServer.LOG.trace("Disconnection {}; # active connections={}", channel.remoteAddress(), + rpcServer.allChannels.size() - 1); + rpcServer.allChannels.remove(channel); }); InetSocketAddress inetSocketAddress = ((InetSocketAddress) channel.remoteAddress()); this.addr = inetSocketAddress.getAddress(); @@ -64,38 +69,23 @@ class NettyServerRpcConnection extends ServerRpcConnection { this.remotePort = inetSocketAddress.getPort(); } - void process(final ByteBuf buf) throws IOException, InterruptedException { - if (connectionHeaderRead) { - this.callCleanup = () -> ReferenceCountUtil.safeRelease(buf); - process(new SingleByteBuff(buf.nioBuffer())); - } else { - ByteBuffer connectionHeader = ByteBuffer.allocate(buf.readableBytes()); - try { - buf.readBytes(connectionHeader); - } finally { - buf.release(); - } - process(connectionHeader); - } - } - - void process(ByteBuffer buf) throws IOException, InterruptedException { - process(new SingleByteBuff(buf)); + void setupHandler() { + channel.pipeline() + .addLast("frameDecoder", new NettyRpcFrameDecoder(rpcServer.maxRequestSize, this)) + .addLast("decoder", new NettyRpcServerRequestDecoder(rpcServer.metrics, this)) + .addLast("encoder", new NettyRpcServerResponseEncoder(rpcServer.metrics)); } - void process(ByteBuff buf) throws IOException, InterruptedException { + void process(ByteBuf buf) throws IOException, InterruptedException { + if (skipInitialSaslHandshake) { + skipInitialSaslHandshake = false; + buf.release(); + return; + } + this.callCleanup = () -> buf.release(); + ByteBuff byteBuff = new SingleByteBuff(buf.nioBuffer()); try { - if (skipInitialSaslHandshake) { - skipInitialSaslHandshake = false; - callCleanupIfNeeded(); - return; - } - - if (useSasl) { - saslReadAndProcess(buf); - } else { - processOneRpc(buf); - } + processOneRpc(byteBuff); } catch (Exception e) { callCleanupIfNeeded(); throw e; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java index e488dc2d4ac0..2c7a8b637414 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java @@ -399,37 +399,6 @@ private static ByteBuffer createHeaderAndMessageBytes(Message result, Message he return pbBuf; } - protected BufferChain wrapWithSasl(BufferChain bc) throws IOException { - if (!this.connection.useSasl) { - return bc; - } - // Looks like no way around this; saslserver wants a byte array. I have to make it one. - // THIS IS A BIG UGLY COPY. - byte[] responseBytes = bc.getBytes(); - byte[] token; - // synchronization may be needed since there can be multiple Handler - // threads using saslServer or Crypto AES to wrap responses. - if (connection.useCryptoAesWrap) { - // wrap with Crypto AES - synchronized (connection.cryptoAES) { - token = connection.cryptoAES.wrap(responseBytes, 0, responseBytes.length); - } - } else { - synchronized (connection.saslServer) { - token = connection.saslServer.wrap(responseBytes, 0, responseBytes.length); - } - } - if (RpcServer.LOG.isTraceEnabled()) { - RpcServer.LOG - .trace("Adding saslServer wrapped token of size " + token.length + " as call response."); - } - - ByteBuffer[] responseBufs = new ByteBuffer[2]; - responseBufs[0] = ByteBuffer.wrap(Bytes.toBytes(token.length)); - responseBufs[1] = ByteBuffer.wrap(token); - return new BufferChain(responseBufs); - } - @Override public long disconnectSince() { if (!this.connection.isConnectionOpen()) { @@ -556,21 +525,7 @@ public int getRemotePort() { @Override public synchronized BufferChain getResponse() { - if (connection.useWrap) { - /* - * wrapping result with SASL as the last step just before sending it out, so every message - * must have the right increasing sequence number - */ - try { - return wrapWithSasl(response); - } catch (IOException e) { - /* it is exactly the same what setResponse() does */ - RpcServer.LOG.warn("Exception while creating response " + e); - return null; - } - } else { - return response; - } + return response; } @RestrictedApi(explanation = "Should only be called in tests", link = "", diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java index 329a8031db27..b09f33c47f9a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java @@ -24,15 +24,12 @@ import io.opentelemetry.context.Context; import io.opentelemetry.context.Scope; import io.opentelemetry.context.propagation.TextMapGetter; -import java.io.ByteArrayInputStream; import java.io.Closeable; import java.io.DataOutputStream; import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.security.GeneralSecurityException; import java.util.Objects; import java.util.Properties; @@ -47,7 +44,6 @@ import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup; import org.apache.hadoop.hbase.nio.ByteBuff; -import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.HBaseSaslRpcServer; import org.apache.hadoop.hbase.security.SaslStatus; @@ -58,7 +54,7 @@ import org.apache.hadoop.hbase.security.provider.SimpleSaslServerAuthenticationProvider; import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; @@ -67,7 +63,6 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.security.authorize.ProxyUsers; -import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; @@ -120,16 +115,9 @@ abstract class ServerRpcConnection implements Closeable { protected BlockingService service; protected SaslServerAuthenticationProvider provider; - protected boolean saslContextEstablished; protected boolean skipInitialSaslHandshake; - private ByteBuffer unwrappedData; - // When is this set? FindBugs wants to know! Says NP - private ByteBuffer unwrappedDataLengthBuffer = ByteBuffer.allocate(4); protected boolean useSasl; protected HBaseSaslRpcServer saslServer; - protected CryptoAES cryptoAES; - protected boolean useWrap = false; - protected boolean useCryptoAesWrap = false; // was authentication allowed with a fallback to simple auth protected boolean authenticatedWithFallback; @@ -164,7 +152,7 @@ public int getRemotePort() { } public VersionInfo getVersionInfo() { - if (connectionHeader.hasVersionInfo()) { + if (connectionHeader != null && connectionHeader.hasVersionInfo()) { return connectionHeader.getVersionInfo(); } return null; @@ -181,18 +169,24 @@ private String getFatalConnectionString(final int version, final byte authByte) /** * Set up cell block codecs */ - private void setupCellBlockCodecs(final ConnectionHeader header) throws FatalConnectionException { + private void setupCellBlockCodecs() throws FatalConnectionException { // TODO: Plug in other supported decoders. - if (!header.hasCellBlockCodecClass()) return; - String className = header.getCellBlockCodecClass(); - if (className == null || className.length() == 0) return; + if (!connectionHeader.hasCellBlockCodecClass()) { + return; + } + String className = connectionHeader.getCellBlockCodecClass(); + if (className == null || className.length() == 0) { + return; + } try { this.codec = (Codec) Class.forName(className).getDeclaredConstructor().newInstance(); } catch (Exception e) { throw new UnsupportedCellCodecException(className, e); } - if (!header.hasCellBlockCompressorClass()) return; - className = header.getCellBlockCompressorClass(); + if (!connectionHeader.hasCellBlockCompressorClass()) { + return; + } + className = connectionHeader.getCellBlockCompressorClass(); try { this.compressionCodec = (CompressionCodec) Class.forName(className).getDeclaredConstructor().newInstance(); @@ -202,21 +196,29 @@ private void setupCellBlockCodecs(final ConnectionHeader header) throws FatalCon } /** - * Set up cipher for rpc encryption with Apache Commons Crypto + * Set up cipher for rpc encryption with Apache Commons Crypto. */ - private void setupCryptoCipher(final ConnectionHeader header, - RPCProtos.ConnectionHeaderResponse.Builder chrBuilder) throws FatalConnectionException { + private Pair setupCryptoCipher() + throws FatalConnectionException { // If simple auth, return - if (saslServer == null) return; + if (saslServer == null) { + return null; + } // check if rpc encryption with Crypto AES String qop = saslServer.getNegotiatedQop(); boolean isEncryption = SaslUtil.QualityOfProtection.PRIVACY.getSaslQop().equalsIgnoreCase(qop); boolean isCryptoAesEncryption = isEncryption && this.rpcServer.conf.getBoolean("hbase.rpc.crypto.encryption.aes.enabled", false); - if (!isCryptoAesEncryption) return; - if (!header.hasRpcCryptoCipherTransformation()) return; - String transformation = header.getRpcCryptoCipherTransformation(); - if (transformation == null || transformation.length() == 0) return; + if (!isCryptoAesEncryption) { + return null; + } + if (!connectionHeader.hasRpcCryptoCipherTransformation()) { + return null; + } + String transformation = connectionHeader.getRpcCryptoCipherTransformation(); + if (transformation == null || transformation.length() == 0) { + return null; + } // Negotiates AES based on complete saslServer. // The Crypto metadata need to be encrypted and send to client. Properties properties = new Properties(); @@ -242,6 +244,7 @@ private void setupCryptoCipher(final ConnectionHeader header, byte[] inIv = new byte[len]; byte[] outIv = new byte[len]; + CryptoAES cryptoAES; try { // generate the cipher meta data with SecureRandom CryptoRandom secureRandom = CryptoRandomFactory.getCryptoRandom(properties); @@ -252,19 +255,20 @@ private void setupCryptoCipher(final ConnectionHeader header, // create CryptoAES for server cryptoAES = new CryptoAES(transformation, properties, inKey, outKey, inIv, outIv); - // create SaslCipherMeta and send to client, - // for client, the [inKey, outKey], [inIv, outIv] should be reversed - RPCProtos.CryptoCipherMeta.Builder ccmBuilder = RPCProtos.CryptoCipherMeta.newBuilder(); - ccmBuilder.setTransformation(transformation); - ccmBuilder.setInIv(getByteString(outIv)); - ccmBuilder.setInKey(getByteString(outKey)); - ccmBuilder.setOutIv(getByteString(inIv)); - ccmBuilder.setOutKey(getByteString(inKey)); - chrBuilder.setCryptoCipherMeta(ccmBuilder); - useCryptoAesWrap = true; } catch (GeneralSecurityException | IOException ex) { throw new UnsupportedCryptoException(ex.getMessage(), ex); } + // create SaslCipherMeta and send to client, + // for client, the [inKey, outKey], [inIv, outIv] should be reversed + RPCProtos.CryptoCipherMeta.Builder ccmBuilder = RPCProtos.CryptoCipherMeta.newBuilder(); + ccmBuilder.setTransformation(transformation); + ccmBuilder.setInIv(getByteString(outIv)); + ccmBuilder.setInKey(getByteString(outKey)); + ccmBuilder.setOutIv(getByteString(inIv)); + ccmBuilder.setOutKey(getByteString(inKey)); + RPCProtos.ConnectionHeaderResponse resp = + RPCProtos.ConnectionHeaderResponse.newBuilder().setCryptoCipherMeta(ccmBuilder).build(); + return Pair.newPair(resp, cryptoAES); } private ByteString getByteString(byte[] bytes) { @@ -327,125 +331,20 @@ protected final void doRawSaslReply(SaslStatus status, Writable rv, String error doRespond(() -> bc); } - public void saslReadAndProcess(ByteBuff saslToken) throws IOException, InterruptedException { - if (saslContextEstablished) { - RpcServer.LOG.trace("Read input token of size={} for processing by saslServer.unwrap()", - saslToken.limit()); - if (!useWrap) { - processOneRpc(saslToken); - } else { - byte[] b = saslToken.hasArray() ? saslToken.array() : saslToken.toBytes(); - byte[] plaintextData; - if (useCryptoAesWrap) { - // unwrap with CryptoAES - plaintextData = cryptoAES.unwrap(b, 0, b.length); - } else { - plaintextData = saslServer.unwrap(b, 0, b.length); - } - // release the request buffer as we have already unwrapped all its content - callCleanupIfNeeded(); - processUnwrappedData(plaintextData); - } - } else { - byte[] replyToken; - try { - if (saslServer == null) { - try { - saslServer = - new HBaseSaslRpcServer(provider, rpcServer.saslProps, rpcServer.secretManager); - } catch (Exception e) { - RpcServer.LOG.error("Error when trying to create instance of HBaseSaslRpcServer " - + "with sasl provider: " + provider, e); - throw e; - } - RpcServer.LOG.debug("Created SASL server with mechanism={}", - provider.getSaslAuthMethod().getAuthMethod()); - } - RpcServer.LOG.debug( - "Read input token of size={} for processing by saslServer." + "evaluateResponse()", - saslToken.limit()); - replyToken = saslServer - .evaluateResponse(saslToken.hasArray() ? saslToken.array() : saslToken.toBytes()); - } catch (IOException e) { - RpcServer.LOG.debug("Failed to execute SASL handshake", e); - IOException sendToClient = e; - Throwable cause = e; - while (cause != null) { - if (cause instanceof InvalidToken) { - sendToClient = (InvalidToken) cause; - break; - } - cause = cause.getCause(); - } - doRawSaslReply(SaslStatus.ERROR, null, sendToClient.getClass().getName(), - sendToClient.getLocalizedMessage()); - this.rpcServer.metrics.authenticationFailure(); - String clientIP = this.toString(); - // attempting user could be null - RpcServer.AUDITLOG.warn("{} {}: {}", RpcServer.AUTH_FAILED_FOR, clientIP, - saslServer.getAttemptingUser()); - throw e; - } finally { - // release the request buffer as we have already unwrapped all its content - callCleanupIfNeeded(); - } - if (replyToken != null) { - if (RpcServer.LOG.isDebugEnabled()) { - RpcServer.LOG.debug("Will send token of size " + replyToken.length + " from saslServer."); - } - doRawSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null, null); - } - if (saslServer.isComplete()) { - String qop = saslServer.getNegotiatedQop(); - useWrap = qop != null && !"auth".equalsIgnoreCase(qop); - ugi = - provider.getAuthorizedUgi(saslServer.getAuthorizationID(), this.rpcServer.secretManager); - RpcServer.LOG.debug( - "SASL server context established. Authenticated client: {}. Negotiated QoP is {}", ugi, - qop); - this.rpcServer.metrics.authenticationSuccess(); - RpcServer.AUDITLOG.info(RpcServer.AUTH_SUCCESSFUL_FOR + ugi); - saslContextEstablished = true; - } + HBaseSaslRpcServer getOrCreateSaslServer() throws IOException { + if (saslServer == null) { + saslServer = new HBaseSaslRpcServer(provider, rpcServer.saslProps, rpcServer.secretManager); } + return saslServer; } - private void processUnwrappedData(byte[] inBuf) throws IOException, InterruptedException { - ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(inBuf)); - // Read all RPCs contained in the inBuf, even partial ones - while (true) { - int count; - if (unwrappedDataLengthBuffer.remaining() > 0) { - count = this.rpcServer.channelRead(ch, unwrappedDataLengthBuffer); - if (count <= 0 || unwrappedDataLengthBuffer.remaining() > 0) { - return; - } - } - - if (unwrappedData == null) { - unwrappedDataLengthBuffer.flip(); - int unwrappedDataLength = unwrappedDataLengthBuffer.getInt(); - - if (unwrappedDataLength == RpcClient.PING_CALL_ID) { - if (RpcServer.LOG.isDebugEnabled()) RpcServer.LOG.debug("Received ping message"); - unwrappedDataLengthBuffer.clear(); - continue; // ping message - } - unwrappedData = ByteBuffer.allocate(unwrappedDataLength); - } - - count = this.rpcServer.channelRead(ch, unwrappedData); - if (count <= 0 || unwrappedData.remaining() > 0) { - return; - } - - if (unwrappedData.remaining() == 0) { - unwrappedDataLengthBuffer.clear(); - unwrappedData.flip(); - processOneRpc(new SingleByteBuff(unwrappedData)); - unwrappedData = null; - } - } + void finishSaslNegotiation() throws IOException { + String qop = saslServer.getNegotiatedQop(); + ugi = provider.getAuthorizedUgi(saslServer.getAuthorizationID(), this.rpcServer.secretManager); + RpcServer.LOG.debug( + "SASL server context established. Authenticated client: {}. Negotiated QoP is {}", ugi, qop); + rpcServer.metrics.authenticationSuccess(); + RpcServer.AUDITLOG.info(RpcServer.AUTH_SUCCESSFUL_FOR + ugi); } public void processOneRpc(ByteBuff buf) throws IOException, InterruptedException { @@ -453,6 +352,7 @@ public void processOneRpc(ByteBuff buf) throws IOException, InterruptedException processRequest(buf); } else { processConnectionHeader(buf); + callCleanupIfNeeded(); this.connectionHeaderRead = true; if (rpcServer.needAuthorization() && !authorizeConnection()) { // Throw FatalConnectionException wrapping ACE so client does right thing and closes @@ -486,25 +386,35 @@ private boolean authorizeConnection() throws IOException { return true; } - // Reads the connection header following version - private void processConnectionHeader(ByteBuff buf) throws IOException { + private CodedInputStream createCis(ByteBuff buf) { + // Here we read in the header. We avoid having pb + // do its default 4k allocation for CodedInputStream. We force it to use + // backing array. + CodedInputStream cis; if (buf.hasArray()) { - this.connectionHeader = ConnectionHeader.parseFrom(buf.array()); + cis = UnsafeByteOperations + .unsafeWrap(buf.array(), buf.arrayOffset() + buf.position(), buf.limit()).newCodedInput(); } else { - CodedInputStream cis = UnsafeByteOperations - .unsafeWrap(new ByteBuffByteInput(buf, 0, buf.limit()), 0, buf.limit()).newCodedInput(); - cis.enableAliasing(true); - this.connectionHeader = ConnectionHeader.parseFrom(cis); + cis = UnsafeByteOperations.unsafeWrap(new ByteBuffByteInput(buf, buf.limit()), 0, buf.limit()) + .newCodedInput(); } + cis.enableAliasing(true); + return cis; + } + + // Reads the connection header following version + private void processConnectionHeader(ByteBuff buf) throws IOException { + this.connectionHeader = ConnectionHeader.parseFrom(createCis(buf)); String serviceName = connectionHeader.getServiceName(); - if (serviceName == null) throw new EmptyServiceNameException(); + if (serviceName == null) { + throw new EmptyServiceNameException(); + } this.service = RpcServer.getService(this.rpcServer.services, serviceName); - if (this.service == null) throw new UnknownServiceException(serviceName); - setupCellBlockCodecs(this.connectionHeader); - RPCProtos.ConnectionHeaderResponse.Builder chrBuilder = - RPCProtos.ConnectionHeaderResponse.newBuilder(); - setupCryptoCipher(this.connectionHeader, chrBuilder); - responseConnectionHeader(chrBuilder); + if (this.service == null) { + throw new UnknownServiceException(serviceName); + } + setupCellBlockCodecs(); + sendConnectionHeaderResponseIfNeeded(); UserGroupInformation protocolUser = createUser(connectionHeader); if (!useSasl) { ugi = protocolUser; @@ -553,25 +463,35 @@ private void processConnectionHeader(ByteBuff buf) throws IOException { /** * Send the response for connection header */ - private void responseConnectionHeader(RPCProtos.ConnectionHeaderResponse.Builder chrBuilder) - throws FatalConnectionException { + private void sendConnectionHeaderResponseIfNeeded() throws FatalConnectionException { + Pair pair = setupCryptoCipher(); // Response the connection header if Crypto AES is enabled - if (!chrBuilder.hasCryptoCipherMeta()) return; + if (pair == null) { + return; + } try { - byte[] connectionHeaderResBytes = chrBuilder.build().toByteArray(); - // encrypt the Crypto AES cipher meta data with sasl server, and send to client - byte[] unwrapped = new byte[connectionHeaderResBytes.length + 4]; - Bytes.putBytes(unwrapped, 0, Bytes.toBytes(connectionHeaderResBytes.length), 0, 4); - Bytes.putBytes(unwrapped, 4, connectionHeaderResBytes, 0, connectionHeaderResBytes.length); - byte[] wrapped = saslServer.wrap(unwrapped, 0, unwrapped.length); + int size = pair.getFirst().getSerializedSize(); BufferChain bc; - try (ByteBufferOutputStream response = new ByteBufferOutputStream(wrapped.length + 4); - DataOutputStream out = new DataOutputStream(response)) { - out.writeInt(wrapped.length); - out.write(wrapped); - bc = new BufferChain(response.getByteBuffer()); + try (ByteBufferOutputStream bbOut = new ByteBufferOutputStream(4 + size); + DataOutputStream out = new DataOutputStream(bbOut)) { + out.writeInt(size); + pair.getFirst().writeTo(out); + bc = new BufferChain(bbOut.getByteBuffer()); } - doRespond(() -> bc); + doRespond(new RpcResponse() { + + @Override + public BufferChain getResponse() { + return bc; + } + + @Override + public void done() { + // must switch after sending the connection header response, as the client still uses the + // original SaslClient to unwrap the data we send back + saslServer.switchToCryptoAES(pair.getSecond()); + } + }); } catch (IOException ex) { throw new UnsupportedCryptoException(ex.getMessage(), ex); } @@ -591,14 +511,7 @@ protected void processRequest(ByteBuff buf) throws IOException, InterruptedExcep // Here we read in the header. We avoid having pb // do its default 4k allocation for CodedInputStream. We force it to use // backing array. - CodedInputStream cis; - if (buf.hasArray()) { - cis = UnsafeByteOperations.unsafeWrap(buf.array(), 0, buf.limit()).newCodedInput(); - } else { - cis = UnsafeByteOperations - .unsafeWrap(new ByteBuffByteInput(buf, 0, buf.limit()), 0, buf.limit()).newCodedInput(); - } - cis.enableAliasing(true); + CodedInputStream cis = createCis(buf); int headerSize = cis.readRawVarint32(); offset = cis.getTotalBytesRead(); Message.Builder builder = RequestHeader.newBuilder(); @@ -739,7 +652,7 @@ private void doBadPreambleHandling(String msg) throws IOException { } private void doBadPreambleHandling(String msg, Exception e) throws IOException { - SimpleRpcServer.LOG.warn(msg); + RpcServer.LOG.warn(msg); doRespond(getErrorResponse(msg, e)); } @@ -764,7 +677,7 @@ protected final boolean processPreamble(ByteBuffer preambleBuffer) throws IOExce int version = preambleBuffer.get() & 0xFF; byte authbyte = preambleBuffer.get(); - if (version != SimpleRpcServer.CURRENT_VERSION) { + if (version != RpcServer.CURRENT_VERSION) { String msg = getFatalConnectionString(version, authbyte); doBadPreambleHandling(msg, new WrongVersionException(msg)); return false; @@ -812,34 +725,28 @@ public abstract ServerCall createCall(int id, BlockingService service, Method private static class ByteBuffByteInput extends ByteInput { private ByteBuff buf; - private int offset; private int length; - ByteBuffByteInput(ByteBuff buf, int offset, int length) { + ByteBuffByteInput(ByteBuff buf, int length) { this.buf = buf; - this.offset = offset; this.length = length; } @Override public byte read(int offset) { - return this.buf.get(getAbsoluteOffset(offset)); - } - - private int getAbsoluteOffset(int offset) { - return this.offset + offset; + return this.buf.get(offset); } @Override public int read(int offset, byte[] out, int outOffset, int len) { - this.buf.get(getAbsoluteOffset(offset), out, outOffset, len); + this.buf.get(offset, out, outOffset, len); return len; } @Override public int read(int offset, ByteBuffer out) { int len = out.remaining(); - this.buf.get(out, getAbsoluteOffset(offset), len); + this.buf.get(out, offset, len); return len; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServerResponder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServerResponder.java index 3043faf609ee..cbf023e2ba9d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServerResponder.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServerResponder.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.channels.CancelledKeyException; import java.nio.channels.ClosedChannelException; import java.nio.channels.SelectionKey; @@ -28,6 +29,8 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.security.HBaseSaslRpcServer; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.util.StringUtils; @@ -217,6 +220,28 @@ private void doAsyncWrite(SelectionKey key) throws IOException { } } + private BufferChain wrapWithSasl(HBaseSaslRpcServer saslServer, BufferChain bc) + throws IOException { + // Looks like no way around this; saslserver wants a byte array. I have to make it one. + // THIS IS A BIG UGLY COPY. + byte[] responseBytes = bc.getBytes(); + byte[] token; + // synchronization may be needed since there can be multiple Handler + // threads using saslServer or Crypto AES to wrap responses. + synchronized (saslServer) { + token = saslServer.wrap(responseBytes, 0, responseBytes.length); + } + if (SimpleRpcServer.LOG.isTraceEnabled()) { + SimpleRpcServer.LOG + .trace("Adding saslServer wrapped token of size " + token.length + " as call response."); + } + + ByteBuffer[] responseBufs = new ByteBuffer[2]; + responseBufs[0] = ByteBuffer.wrap(Bytes.toBytes(token.length)); + responseBufs[1] = ByteBuffer.wrap(token); + return new BufferChain(responseBufs); + } + /** * Process the response for this call. You need to have the lock on * {@link org.apache.hadoop.hbase.ipc.SimpleServerRpcConnection#responseWriteLock} @@ -226,6 +251,9 @@ private boolean processResponse(SimpleServerRpcConnection conn, RpcResponse resp throws IOException { boolean error = true; BufferChain buf = resp.getResponse(); + if (conn.useWrap) { + buf = wrapWithSasl(conn.saslServer, buf); + } try { // Send as much data as we can in the non-blocking fashion long numBytes = this.simpleRpcServer.channelWrite(conn.channel, buf); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java index d7c4b0317aca..ac705d7a26fa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java @@ -17,11 +17,13 @@ */ package org.apache.hadoop.hbase.ipc; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.net.InetAddress; import java.net.Socket; import java.nio.ByteBuffer; +import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; import java.nio.channels.SocketChannel; import java.util.concurrent.ConcurrentLinkedDeque; @@ -34,7 +36,11 @@ import org.apache.hadoop.hbase.exceptions.RequestTooBigException; import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup; import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.hadoop.hbase.nio.SingleByteBuff; +import org.apache.hadoop.hbase.security.HBaseSaslRpcServer; +import org.apache.hadoop.hbase.security.SaslStatus; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.io.BytesWritable; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; @@ -63,6 +69,11 @@ class SimpleServerRpcConnection extends ServerRpcConnection { // If initial preamble with version and magic has been read or not. private boolean connectionPreambleRead = false; + private boolean saslContextEstablished; + private ByteBuffer unwrappedData; + // When is this set? FindBugs wants to know! Says NP + private ByteBuffer unwrappedDataLengthBuffer = ByteBuffer.allocate(4); + boolean useWrap = false; final ConcurrentLinkedDeque responseQueue = new ConcurrentLinkedDeque<>(); final Lock responseWriteLock = new ReentrantLock(); @@ -142,6 +153,110 @@ private int read4Bytes() throws IOException { } } + private void processUnwrappedData(byte[] inBuf) throws IOException, InterruptedException { + ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(inBuf)); + // Read all RPCs contained in the inBuf, even partial ones + while (true) { + int count; + if (unwrappedDataLengthBuffer.remaining() > 0) { + count = this.rpcServer.channelRead(ch, unwrappedDataLengthBuffer); + if (count <= 0 || unwrappedDataLengthBuffer.remaining() > 0) { + return; + } + } + + if (unwrappedData == null) { + unwrappedDataLengthBuffer.flip(); + int unwrappedDataLength = unwrappedDataLengthBuffer.getInt(); + + if (unwrappedDataLength == RpcClient.PING_CALL_ID) { + if (RpcServer.LOG.isDebugEnabled()) RpcServer.LOG.debug("Received ping message"); + unwrappedDataLengthBuffer.clear(); + continue; // ping message + } + unwrappedData = ByteBuffer.allocate(unwrappedDataLength); + } + + count = this.rpcServer.channelRead(ch, unwrappedData); + if (count <= 0 || unwrappedData.remaining() > 0) { + return; + } + + if (unwrappedData.remaining() == 0) { + unwrappedDataLengthBuffer.clear(); + unwrappedData.flip(); + processOneRpc(new SingleByteBuff(unwrappedData)); + unwrappedData = null; + } + } + } + + private void saslReadAndProcess(ByteBuff saslToken) throws IOException, InterruptedException { + if (saslContextEstablished) { + RpcServer.LOG.trace("Read input token of size={} for processing by saslServer.unwrap()", + saslToken.limit()); + if (!useWrap) { + processOneRpc(saslToken); + } else { + byte[] b = saslToken.hasArray() ? saslToken.array() : saslToken.toBytes(); + byte[] plaintextData = saslServer.unwrap(b, 0, b.length); + // release the request buffer as we have already unwrapped all its content + callCleanupIfNeeded(); + processUnwrappedData(plaintextData); + } + } else { + byte[] replyToken; + try { + try { + getOrCreateSaslServer(); + } catch (Exception e) { + RpcServer.LOG.error("Error when trying to create instance of HBaseSaslRpcServer " + + "with sasl provider: " + provider, e); + throw e; + } + RpcServer.LOG.debug("Created SASL server with mechanism={}", + provider.getSaslAuthMethod().getAuthMethod()); + RpcServer.LOG.debug( + "Read input token of size={} for processing by saslServer." + "evaluateResponse()", + saslToken.limit()); + replyToken = saslServer + .evaluateResponse(saslToken.hasArray() ? saslToken.array() : saslToken.toBytes()); + } catch (IOException e) { + RpcServer.LOG.debug("Failed to execute SASL handshake", e); + Throwable sendToClient = HBaseSaslRpcServer.unwrap(e); + doRawSaslReply(SaslStatus.ERROR, null, sendToClient.getClass().getName(), + sendToClient.getLocalizedMessage()); + this.rpcServer.metrics.authenticationFailure(); + String clientIP = this.toString(); + // attempting user could be null + RpcServer.AUDITLOG.warn("{}{}: {}", RpcServer.AUTH_FAILED_FOR, clientIP, + saslServer.getAttemptingUser()); + throw e; + } finally { + // release the request buffer as we have already unwrapped all its content + callCleanupIfNeeded(); + } + if (replyToken != null) { + if (RpcServer.LOG.isDebugEnabled()) { + RpcServer.LOG.debug("Will send token of size " + replyToken.length + " from saslServer."); + } + doRawSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null, null); + } + if (saslServer.isComplete()) { + String qop = saslServer.getNegotiatedQop(); + useWrap = qop != null && !"auth".equalsIgnoreCase(qop); + ugi = + provider.getAuthorizedUgi(saslServer.getAuthorizationID(), this.rpcServer.secretManager); + RpcServer.LOG.debug( + "SASL server context established. Authenticated client: {}. Negotiated QoP is {}", ugi, + qop); + this.rpcServer.metrics.authenticationSuccess(); + RpcServer.AUDITLOG.info(RpcServer.AUTH_SUCCESSFUL_FOR + ugi); + saslContextEstablished = true; + } + } + } + /** * Read off the wire. If there is not enough data to read, update the connection state with what * we have and returns. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java index eb9913174e23..6d375e0014ae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java @@ -24,6 +24,7 @@ import javax.security.sasl.Sasl; import javax.security.sasl.SaslException; import javax.security.sasl.SaslServer; +import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; import org.apache.hadoop.hbase.security.provider.AttemptingUserProvidingSaslServer; import org.apache.hadoop.hbase.security.provider.SaslServerAuthenticationProvider; import org.apache.hadoop.security.token.SecretManager; @@ -40,6 +41,7 @@ public class HBaseSaslRpcServer { private final AttemptingUserProvidingSaslServer serverWithProvider; private final SaslServer saslServer; + private CryptoAES cryptoAES; public HBaseSaslRpcServer(SaslServerAuthenticationProvider provider, Map saslProps, SecretManager secretManager) @@ -61,16 +63,28 @@ public void dispose() { SaslUtil.safeDispose(saslServer); } + public void switchToCryptoAES(CryptoAES cryptoAES) { + this.cryptoAES = cryptoAES; + } + public String getAttemptingUser() { return serverWithProvider.getAttemptingUser().map(Object::toString).orElse("Unknown"); } public byte[] wrap(byte[] buf, int off, int len) throws SaslException { - return saslServer.wrap(buf, off, len); + if (cryptoAES != null) { + return cryptoAES.wrap(buf, off, len); + } else { + return saslServer.wrap(buf, off, len); + } } public byte[] unwrap(byte[] buf, int off, int len) throws SaslException { - return saslServer.unwrap(buf, off, len); + if (cryptoAES != null) { + return cryptoAES.unwrap(buf, off, len); + } else { + return saslServer.unwrap(buf, off, len); + } } public String getNegotiatedQop() { @@ -92,4 +106,18 @@ public static T getIdentifier(String id, } return tokenIdentifier; } + + /** + * Unwrap InvalidToken exception, otherwise return the one passed in. + */ + public static Throwable unwrap(Throwable e) { + Throwable cause = e; + while (cause != null) { + if (cause instanceof InvalidToken) { + return cause; + } + cause = cause.getCause(); + } + return e; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java index ee8cf04e94b5..fe947d33110d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java @@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MatcherPredicate; -import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.util.Bytes; @@ -103,7 +102,7 @@ public abstract class AbstractTestIPC { CONF.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, SimpleRpcServer.class.getName()); } - protected abstract RpcServer createRpcServer(final Server server, final String name, + protected abstract RpcServer createRpcServer(final String name, final List services, final InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException; @@ -117,11 +116,11 @@ protected abstract RpcServer createRpcServer(final Server server, final String n */ @Test public void testNoCodec() throws IOException, ServiceException { - Configuration conf = HBaseConfiguration.create(); - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClientNoCodec(conf)) { + try (AbstractRpcClient client = createRpcClientNoCodec(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); HBaseRpcController pcrc = new HBaseRpcControllerImpl(); @@ -143,18 +142,18 @@ public void testNoCodec() throws IOException, ServiceException { */ @Test public void testCompressCellBlock() throws IOException, ServiceException { - Configuration conf = new Configuration(HBaseConfiguration.create()); - conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName()); + Configuration clientConf = new Configuration(CONF); + clientConf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName()); List cells = new ArrayList<>(); int count = 3; for (int i = 0; i < count; i++) { cells.add(CELL); } - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(conf)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); HBaseRpcController pcrc = new HBaseRpcControllerImpl(CellUtil.createCellScanner(cells)); @@ -179,11 +178,11 @@ public void testCompressCellBlock() throws IOException, ServiceException { @Test public void testRTEDuringConnectionSetup() throws Exception { - Configuration conf = HBaseConfiguration.create(); - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClientRTEDuringConnectionSetup(conf)) { + try (AbstractRpcClient client = createRpcClientRTEDuringConnectionSetup(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); stub.ping(null, EmptyRequestProto.getDefaultInstance()); @@ -201,12 +200,13 @@ public void testRTEDuringConnectionSetup() throws Exception { */ @Test public void testRpcScheduler() throws IOException, ServiceException, InterruptedException { + Configuration clientConf = new Configuration(CONF); RpcScheduler scheduler = spy(new FifoRpcScheduler(CONF, 1)); - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, scheduler); verify(scheduler).init(any(RpcScheduler.Context.class)); - try (AbstractRpcClient client = createRpcClient(CONF)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); verify(scheduler).start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); @@ -224,12 +224,12 @@ public void testRpcScheduler() throws IOException, ServiceException, Interrupted /** Tests that the rpc scheduler is called when requests arrive. */ @Test public void testRpcMaxRequestSize() throws IOException, ServiceException { - Configuration conf = new Configuration(CONF); - conf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000); - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + clientConf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), - new InetSocketAddress("localhost", 0), conf, new FifoRpcScheduler(conf, 1)); - try (AbstractRpcClient client = createRpcClient(conf)) { + new InetSocketAddress("localhost", 0), clientConf, new FifoRpcScheduler(clientConf, 1)); + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); StringBuilder message = new StringBuilder(1200); @@ -258,11 +258,12 @@ public void testRpcMaxRequestSize() throws IOException, ServiceException { @Test public void testRpcServerForNotNullRemoteAddressInCallObject() throws IOException, ServiceException { - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); InetSocketAddress localAddr = new InetSocketAddress("localhost", 0); - try (AbstractRpcClient client = createRpcClient(CONF)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); assertEquals(localAddr.getAddress().getHostAddress(), @@ -274,10 +275,11 @@ public void testRpcServerForNotNullRemoteAddressInCallObject() @Test public void testRemoteError() throws IOException, ServiceException { - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(CONF)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); stub.error(null, EmptyRequestProto.getDefaultInstance()); @@ -293,10 +295,11 @@ public void testRemoteError() throws IOException, ServiceException { @Test public void testTimeout() throws IOException { - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(CONF)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); HBaseRpcController pcrc = new HBaseRpcControllerImpl(); @@ -323,19 +326,19 @@ public void testTimeout() throws IOException { } } - protected abstract RpcServer createTestFailingRpcServer(final Server server, final String name, + protected abstract RpcServer createTestFailingRpcServer(final String name, final List services, final InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException; /** Tests that the connection closing is handled by the client with outstanding RPC calls */ @Test public void testConnectionCloseWithOutstandingRPCs() throws InterruptedException, IOException { - Configuration conf = new Configuration(CONF); - RpcServer rpcServer = createTestFailingRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createTestFailingRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(conf)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); @@ -350,11 +353,11 @@ public void testConnectionCloseWithOutstandingRPCs() throws InterruptedException @Test public void testAsyncEcho() throws IOException { - Configuration conf = HBaseConfiguration.create(); - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(conf)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); Interface stub = newStub(client, rpcServer.getListenerAddress()); int num = 10; @@ -368,10 +371,11 @@ public void testAsyncEcho() throws IOException { callbackList.add(done); } for (int i = 0; i < num; i++) { + EchoResponseProto resp = callbackList.get(i).get(); HBaseRpcController pcrc = pcrcList.get(i); + assertEquals("hello-" + i, resp.getMessage()); assertFalse(pcrc.failed()); assertNull(pcrc.cellScanner()); - assertEquals("hello-" + i, callbackList.get(i).get().getMessage()); } } finally { rpcServer.stop(); @@ -380,8 +384,9 @@ public void testAsyncEcho() throws IOException { @Test public void testAsyncRemoteError() throws IOException { - AbstractRpcClient client = createRpcClient(CONF); - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + AbstractRpcClient client = createRpcClient(clientConf); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); try { @@ -404,10 +409,11 @@ public void testAsyncRemoteError() throws IOException { @Test public void testAsyncTimeout() throws IOException { - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(CONF)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); Interface stub = newStub(client, rpcServer.getListenerAddress()); List pcrcList = new ArrayList<>(); @@ -485,10 +491,11 @@ private void assertRemoteSpan() { @Test public void testTracingSuccessIpc() throws IOException, ServiceException { - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(CONF)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); stub.pause(null, PauseRequestProto.newBuilder().setMs(100).build()); @@ -513,10 +520,11 @@ public void testTracingSuccessIpc() throws IOException, ServiceException { @Test public void testTracingErrorIpc() throws IOException { - RpcServer rpcServer = createRpcServer(null, "testRpcServer", + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); - try (AbstractRpcClient client = createRpcClient(CONF)) { + try (AbstractRpcClient client = createRpcClient(clientConf)) { rpcServer.start(); BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); // use the ISA from the running server so that we can get the port selected. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/FailingNettyRpcServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/FailingNettyRpcServer.java new file mode 100644 index 000000000000..d5c408c23874 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/FailingNettyRpcServer.java @@ -0,0 +1,60 @@ +/* + * 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 java.io.IOException; +import java.net.InetSocketAddress; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.nio.ByteBuff; + +import org.apache.hbase.thirdparty.io.netty.channel.Channel; + +public class FailingNettyRpcServer extends NettyRpcServer { + + public FailingNettyRpcServer(Server server, String name, + List services, InetSocketAddress bindAddress, + Configuration conf, RpcScheduler scheduler) throws IOException { + super(server, name, services, bindAddress, conf, scheduler, true); + } + + static final class FailingConnection extends NettyServerRpcConnection { + private FailingConnection(FailingNettyRpcServer rpcServer, Channel channel) { + super(rpcServer, channel); + } + + @Override + public void processRequest(ByteBuff buf) throws IOException, InterruptedException { + // this will throw exception after the connection header is read, and an RPC is sent + // from client + throw new DoNotRetryIOException("Failing for test"); + } + } + + @Override + protected NettyRpcServerPreambleHandler createNettyRpcServerPreambleHandler() { + return new NettyRpcServerPreambleHandler(FailingNettyRpcServer.this) { + @Override + protected NettyServerRpcConnection createNettyServerRpcConnection(Channel channel) { + return new FailingConnection(FailingNettyRpcServer.this, channel); + } + }; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java index bb71ac2611a5..4d7d0996fabd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java @@ -40,17 +40,17 @@ public class TestBlockingIPC extends AbstractTestIPC { HBaseClassTestRule.forClass(TestBlockingIPC.class); @Override - protected RpcServer createRpcServer(Server server, String name, + protected RpcServer createRpcServer(String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return RpcServerFactory.createRpcServer(server, name, services, bindAddress, conf, scheduler); + return RpcServerFactory.createRpcServer(null, name, services, bindAddress, conf, scheduler); } @Override protected BlockingRpcClient createRpcClientNoCodec(Configuration conf) { return new BlockingRpcClient(conf) { @Override - Codec getCodec() { + protected Codec getCodec() { return null; } }; @@ -67,7 +67,7 @@ protected BlockingRpcClient createRpcClientRTEDuringConnectionSetup(Configuratio return new BlockingRpcClient(conf) { @Override - boolean isTcpNoDelay() { + protected boolean isTcpNoDelay() { throw new RuntimeException("Injected fault"); } }; @@ -102,9 +102,9 @@ protected SimpleServerRpcConnection getConnection(SocketChannel channel, long ti } @Override - protected RpcServer createTestFailingRpcServer(Server server, String name, + protected RpcServer createTestFailingRpcServer(String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return new TestFailingRpcServer(server, name, services, bindAddress, conf, scheduler); + return new TestFailingRpcServer(null, name, services, bindAddress, conf, scheduler); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java index 1cffa7aaf6c1..265ae7852f02 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java @@ -23,11 +23,8 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.util.JVM; @@ -40,7 +37,6 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; -import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollSocketChannel; import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; @@ -107,10 +103,10 @@ private void setConf(Configuration conf) { } @Override - protected RpcServer createRpcServer(Server server, String name, + protected RpcServer createRpcServer(String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return new NettyRpcServer(server, name, services, bindAddress, conf, scheduler, true); + return new NettyRpcServer(null, name, services, bindAddress, conf, scheduler, true); } @Override @@ -119,7 +115,7 @@ protected NettyRpcClient createRpcClientNoCodec(Configuration conf) { return new NettyRpcClient(conf) { @Override - Codec getCodec() { + protected Codec getCodec() { return null; } @@ -138,48 +134,16 @@ protected NettyRpcClient createRpcClientRTEDuringConnectionSetup(Configuration c return new NettyRpcClient(conf) { @Override - boolean isTcpNoDelay() { + protected boolean isTcpNoDelay() { throw new RuntimeException("Injected fault"); } }; } - private static class TestFailingRpcServer extends NettyRpcServer { - - TestFailingRpcServer(Server server, String name, - List services, InetSocketAddress bindAddress, - Configuration conf, RpcScheduler scheduler) throws IOException { - super(server, name, services, bindAddress, conf, scheduler, true); - } - - static final class FailingConnection extends NettyServerRpcConnection { - private FailingConnection(TestFailingRpcServer rpcServer, Channel channel) { - super(rpcServer, channel); - } - - @Override - public void processRequest(ByteBuff buf) throws IOException, InterruptedException { - // this will throw exception after the connection header is read, and an RPC is sent - // from client - throw new DoNotRetryIOException("Failing for test"); - } - } - - @Override - protected NettyRpcServerPreambleHandler createNettyRpcServerPreambleHandler() { - return new NettyRpcServerPreambleHandler(TestFailingRpcServer.this) { - @Override - protected NettyServerRpcConnection createNettyServerRpcConnection(Channel channel) { - return new FailingConnection(TestFailingRpcServer.this, channel); - } - }; - } - } - @Override - protected RpcServer createTestFailingRpcServer(Server server, String name, + protected RpcServer createTestFailingRpcServer(String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return new TestFailingRpcServer(server, name, services, bindAddress, conf, scheduler); + return new FailingNettyRpcServer(null, name, services, bindAddress, conf, scheduler); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestMutualTls.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestMutualTls.java new file mode 100644 index 000000000000..1663d41a67ac --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestMutualTls.java @@ -0,0 +1,225 @@ +/* + * 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.security; + +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThrows; + +import java.io.File; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.net.InetSocketAddress; +import java.security.GeneralSecurityException; +import java.security.Security; +import java.security.cert.X509Certificate; +import javax.net.ssl.SSLHandshakeException; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.io.crypto.tls.KeyStoreFileType; +import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContext; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContextProvider; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcServer; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl; +import org.bouncycastle.asn1.x500.X500NameBuilder; +import org.bouncycastle.asn1.x500.style.BCStyle; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.operator.OperatorCreationException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runners.Parameterized; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos; + +public abstract class AbstractTestMutualTls { + protected static HBaseCommonTestingUtility UTIL; + + protected static File DIR; + + protected static X509TestContextProvider PROVIDER; + + private X509TestContext x509TestContext; + + protected RpcServer rpcServer; + + protected RpcClient rpcClient; + private TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub; + + @Parameterized.Parameter(0) + public X509KeyType caKeyType; + + @Parameterized.Parameter(1) + public X509KeyType certKeyType; + + @Parameterized.Parameter(2) + public String keyPassword; + @Parameterized.Parameter(3) + public boolean expectSuccess; + + @Parameterized.Parameter(4) + public boolean validateHostnames; + + @Parameterized.Parameter(5) + public CertConfig certConfig; + + public enum CertConfig { + // For no cert, we literally pass no certificate to the server. It's possible (assuming server + // allows it based on ClientAuth mode) to use SSL without a KeyStore which will still do all + // the handshaking but without a client cert. This is what we do here. + // This mode only makes sense for client side, as server side must return a cert. + NO_CLIENT_CERT, + // For non-verifiable cert, we create a new certificate which is signed by a different + // CA. So we're passing a cert, but the client/server can't verify it. + NON_VERIFIABLE_CERT, + // Good cert is the default mode, which uses a cert signed by the same CA both sides + // and the hostname should match (localhost) + GOOD_CERT, + // For good cert/bad host, we create a new certificate signed by the same CA. But + // this cert has a SANS that will not match the localhost peer. + VERIFIABLE_CERT_WITH_BAD_HOST + } + + @BeforeClass + public static void setUpBeforeClass() throws IOException { + UTIL = new HBaseCommonTestingUtility(); + Security.addProvider(new BouncyCastleProvider()); + DIR = + new File(UTIL.getDataTestDir(AbstractTestTlsRejectPlainText.class.getSimpleName()).toString()) + .getCanonicalFile(); + FileUtils.forceMkdir(DIR); + Configuration conf = UTIL.getConfiguration(); + conf.setClass(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, NettyRpcClient.class, + RpcClient.class); + conf.setClass(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, NettyRpcServer.class, + RpcServer.class); + conf.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_ENABLED, true); + conf.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, false); + conf.setBoolean(X509Util.HBASE_CLIENT_NETTY_TLS_ENABLED, true); + PROVIDER = new X509TestContextProvider(conf, DIR); + } + + @AfterClass + public static void cleanUp() { + Security.removeProvider(BouncyCastleProvider.PROVIDER_NAME); + UTIL.cleanupTestDir(); + } + + protected abstract void initialize(Configuration serverConf, Configuration clientConf) + throws IOException, GeneralSecurityException, OperatorCreationException; + + @Before + public void setUp() throws Exception { + x509TestContext = PROVIDER.get(caKeyType, certKeyType, keyPassword.toCharArray()); + x509TestContext.setConfigurations(KeyStoreFileType.JKS, KeyStoreFileType.JKS); + + Configuration serverConf = new Configuration(UTIL.getConfiguration()); + Configuration clientConf = new Configuration(UTIL.getConfiguration()); + + initialize(serverConf, clientConf); + + rpcServer = new NettyRpcServer(null, "testRpcServer", + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), + new InetSocketAddress("localhost", 0), serverConf, new FifoRpcScheduler(serverConf, 1), true); + rpcServer.start(); + + rpcClient = new NettyRpcClient(clientConf); + stub = TestProtobufRpcServiceImpl.newBlockingStub(rpcClient, rpcServer.getListenerAddress()); + } + + protected void handleCertConfig(Configuration confToSet) + throws GeneralSecurityException, IOException, OperatorCreationException { + switch (certConfig) { + case NO_CLIENT_CERT: + // clearing out the keystore location will cause no cert to be sent. + confToSet.set(X509Util.TLS_CONFIG_KEYSTORE_LOCATION, ""); + break; + case NON_VERIFIABLE_CERT: + // to simulate a bad cert, we inject a new keystore into the client side. + // the same truststore exists, so it will still successfully verify the server cert + // but since the new client keystore cert is created from a new CA (which the server doesn't + // have), + // the server will not be able to verify it. + X509TestContext context = + PROVIDER.get(caKeyType, certKeyType, "random value".toCharArray()); + context.setKeystoreConfigurations(KeyStoreFileType.JKS, confToSet); + break; + case VERIFIABLE_CERT_WITH_BAD_HOST: + // to simulate a good cert with a bad host, we need to create a new cert using the existing + // context's CA/truststore. Here we can pass any random SANS, as long as it won't match + // localhost or any reasonable name that this test might run on. + X509Certificate cert = x509TestContext.newCert(new X500NameBuilder(BCStyle.INSTANCE) + .addRDN(BCStyle.CN, + MethodHandles.lookup().lookupClass().getCanonicalName() + " With Bad Host Test") + .build(), "www.example.com"); + x509TestContext.cloneWithNewKeystoreCert(cert) + .setKeystoreConfigurations(KeyStoreFileType.JKS, confToSet); + break; + default: + break; + } + } + + @After + public void tearDown() throws IOException { + if (rpcServer != null) { + rpcServer.stop(); + } + Closeables.close(rpcClient, true); + x509TestContext.clearConfigurations(); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_OCSP); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_CLR); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_PROTOCOL); + System.clearProperty("com.sun.net.ssl.checkRevocation"); + System.clearProperty("com.sun.security.enableCRLDP"); + Security.setProperty("ocsp.enable", Boolean.FALSE.toString()); + Security.setProperty("com.sun.security.enableCRLDP", Boolean.FALSE.toString()); + } + + @Test + public void testClientAuth() throws Exception { + if (expectSuccess) { + // we expect no exception, so if one is thrown the test will fail + submitRequest(); + } else { + ServiceException se = assertThrows(ServiceException.class, this::submitRequest); + assertThat(se.getCause(), instanceOf(SSLHandshakeException.class)); + } + } + + private void submitRequest() throws ServiceException { + stub.echo(null, TestProtos.EchoRequestProto.newBuilder().setMessage("hello world").build()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java new file mode 100644 index 000000000000..a5bca1a0ccf0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java @@ -0,0 +1,390 @@ +/* + * 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.security; + +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.loginKerberosPrincipal; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.setSecuredConfiguration; +import static org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProviders.SELECTOR_KEY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.fail; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.ipc.RpcServerInterface; +import org.apache.hadoop.hbase.security.provider.AuthenticationProviderSelector; +import org.apache.hadoop.hbase.security.provider.BuiltInProviderSelector; +import org.apache.hadoop.hbase.security.provider.SaslAuthMethod; +import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProvider; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.minikdc.MiniKdc; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation; + +public class AbstractTestSecureIPC { + + protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + protected static final File KEYTAB_FILE = + new File(TEST_UTIL.getDataTestDir("keytab").toUri().getPath()); + + protected static MiniKdc KDC; + protected static String HOST = "localhost"; + protected static String PRINCIPAL; + + protected String krbKeytab; + protected String krbPrincipal; + protected UserGroupInformation ugi; + protected Configuration clientConf; + protected Configuration serverConf; + + protected static void initKDCAndConf() throws Exception { + KDC = TEST_UTIL.setupMiniKdc(KEYTAB_FILE); + PRINCIPAL = "hbase/" + HOST; + KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL); + HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm()); + // set a smaller timeout and retry to speed up tests + TEST_UTIL.getConfiguration().setInt(RpcClient.SOCKET_TIMEOUT_READ, 2000); + TEST_UTIL.getConfiguration().setInt("hbase.security.relogin.maxretries", 1); + } + + protected static void stopKDC() throws InterruptedException { + if (KDC != null) { + KDC.stop(); + } + } + + protected final void setUpPrincipalAndConf() throws Exception { + krbKeytab = getKeytabFileForTesting(); + krbPrincipal = getPrincipalForTesting(); + ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal); + clientConf = new Configuration(TEST_UTIL.getConfiguration()); + setSecuredConfiguration(clientConf); + serverConf = new Configuration(TEST_UTIL.getConfiguration()); + setSecuredConfiguration(serverConf); + } + + @Test + public void testRpcCallWithEnabledKerberosSaslAuth() throws Exception { + UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); + + // check that the login user is okay: + assertSame(ugi2, ugi); + assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); + assertEquals(krbPrincipal, ugi.getUserName()); + + callRpcService(User.create(ugi2)); + } + + @Test + public void testRpcCallWithEnabledKerberosSaslAuthCanonicalHostname() throws Exception { + UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); + + // check that the login user is okay: + assertSame(ugi2, ugi); + assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); + assertEquals(krbPrincipal, ugi.getUserName()); + + enableCanonicalHostnameTesting(clientConf, "localhost"); + clientConf.setBoolean( + SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS, false); + clientConf.set(HBaseKerberosUtils.KRB_PRINCIPAL, "hbase/_HOST@" + KDC.getRealm()); + + callRpcService(User.create(ugi2)); + } + + @Test + public void testRpcCallWithEnabledKerberosSaslAuthNoCanonicalHostname() throws Exception { + UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); + + // check that the login user is okay: + assertSame(ugi2, ugi); + assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); + assertEquals(krbPrincipal, ugi.getUserName()); + + enableCanonicalHostnameTesting(clientConf, "127.0.0.1"); + clientConf + .setBoolean(SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS, true); + clientConf.set(HBaseKerberosUtils.KRB_PRINCIPAL, "hbase/_HOST@" + KDC.getRealm()); + + callRpcService(User.create(ugi2)); + } + + private static void enableCanonicalHostnameTesting(Configuration conf, String canonicalHostname) { + conf.setClass(SELECTOR_KEY, CanonicalHostnameTestingAuthenticationProviderSelector.class, + AuthenticationProviderSelector.class); + conf.set(CanonicalHostnameTestingAuthenticationProviderSelector.CANONICAL_HOST_NAME_KEY, + canonicalHostname); + } + + public static class CanonicalHostnameTestingAuthenticationProviderSelector + extends BuiltInProviderSelector { + private static final String CANONICAL_HOST_NAME_KEY = + "CanonicalHostnameTestingAuthenticationProviderSelector.canonicalHostName"; + + @Override + public Pair> + selectProvider(String clusterId, User user) { + final Pair> pair = + super.selectProvider(clusterId, user); + pair.setFirst(createCanonicalHostNameTestingProvider(pair.getFirst())); + return pair; + } + + SaslClientAuthenticationProvider + createCanonicalHostNameTestingProvider(SaslClientAuthenticationProvider delegate) { + return new SaslClientAuthenticationProvider() { + @Override + public SaslClient createClient(Configuration conf, InetAddress serverAddr, + SecurityInfo securityInfo, Token token, + boolean fallbackAllowed, Map saslProps) throws IOException { + final String s = conf.get(CANONICAL_HOST_NAME_KEY); + if (s != null) { + try { + final Field canonicalHostName = + InetAddress.class.getDeclaredField("canonicalHostName"); + canonicalHostName.setAccessible(true); + canonicalHostName.set(serverAddr, s); + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + return delegate.createClient(conf, serverAddr, securityInfo, token, fallbackAllowed, + saslProps); + } + + @Override + public UserInformation getUserInfo(User user) { + return delegate.getUserInfo(user); + } + + @Override + public UserGroupInformation getRealUser(User ugi) { + return delegate.getRealUser(ugi); + } + + @Override + public boolean canRetry() { + return delegate.canRetry(); + } + + @Override + public void relogin() throws IOException { + delegate.relogin(); + } + + @Override + public SaslAuthMethod getSaslAuthMethod() { + return delegate.getSaslAuthMethod(); + } + + @Override + public String getTokenKind() { + return delegate.getTokenKind(); + } + }; + } + } + + @Test + public void testRpcFallbackToSimpleAuth() throws Exception { + String clientUsername = "testuser"; + UserGroupInformation clientUgi = + UserGroupInformation.createUserForTesting(clientUsername, new String[] { clientUsername }); + + // check that the client user is insecure + assertNotSame(ugi, clientUgi); + assertEquals(AuthenticationMethod.SIMPLE, clientUgi.getAuthenticationMethod()); + assertEquals(clientUsername, clientUgi.getUserName()); + + clientConf.set(User.HBASE_SECURITY_CONF_KEY, "simple"); + serverConf.setBoolean(RpcServer.FALLBACK_TO_INSECURE_CLIENT_AUTH, true); + callRpcService(User.create(clientUgi)); + } + + private void setRpcProtection(String clientProtection, String serverProtection) { + clientConf.set("hbase.rpc.protection", clientProtection); + serverConf.set("hbase.rpc.protection", serverProtection); + } + + /** + * Test various combinations of Server and Client qops. + */ + @Test + public void testSaslWithCommonQop() throws Exception { + setRpcProtection("privacy,authentication", "authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("authentication", "privacy,authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("integrity,authentication", "privacy,authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("integrity,authentication", "integrity,authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("privacy,authentication", "privacy,authentication"); + callRpcService(User.create(ugi)); + } + + @Test + public void testSaslNoCommonQop() throws Exception { + setRpcProtection("integrity", "privacy"); + SaslException se = assertThrows(SaslException.class, () -> callRpcService(User.create(ugi))); + assertEquals("No common protection layer between client and server", se.getMessage()); + } + + /** + * Test sasl encryption with Crypto AES. + */ + @Test + public void testSaslWithCryptoAES() throws Exception { + setRpcProtection("privacy", "privacy"); + setCryptoAES("true", "true"); + callRpcService(User.create(ugi)); + } + + /** + * Test various combinations of Server and Client configuration for Crypto AES. n + */ + @Test + public void testDifferentConfWithCryptoAES() throws Exception { + setRpcProtection("privacy", "privacy"); + + setCryptoAES("false", "true"); + callRpcService(User.create(ugi)); + + setCryptoAES("true", "false"); + try { + callRpcService(User.create(ugi)); + fail("The exception should be thrown out for the rpc timeout."); + } catch (Exception e) { + // ignore the expected exception + } + } + + private void setCryptoAES(String clientCryptoAES, String serverCryptoAES) { + clientConf.set("hbase.rpc.crypto.encryption.aes.enabled", clientCryptoAES); + serverConf.set("hbase.rpc.crypto.encryption.aes.enabled", serverCryptoAES); + } + + /** + * Sets up a RPC Server and a Client. Does a RPC checks the result. If an exception is thrown from + * the stub, this function will throw root cause of that exception. + */ + private void callRpcService(User clientUser) throws Exception { + SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class); + Mockito.when(securityInfoMock.getServerPrincipal()) + .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL); + SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock); + + InetSocketAddress isa = new InetSocketAddress(HOST, 0); + + RpcServerInterface rpcServer = RpcServerFactory.createRpcServer(null, "AbstractTestSecureIPC", + Lists + .newArrayList(new RpcServer.BlockingServiceAndInterface((BlockingService) SERVICE, null)), + isa, serverConf, new FifoRpcScheduler(serverConf, 1)); + rpcServer.start(); + try (RpcClient rpcClient = + RpcClientFactory.createClient(clientConf, HConstants.DEFAULT_CLUSTER_ID.toString())) { + BlockingInterface stub = + newBlockingStub(rpcClient, rpcServer.getListenerAddress(), clientUser); + TestThread th1 = new TestThread(stub); + final Throwable exception[] = new Throwable[1]; + Collections.synchronizedList(new ArrayList()); + Thread.UncaughtExceptionHandler exceptionHandler = new Thread.UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread th, Throwable ex) { + exception[0] = ex; + } + }; + th1.setUncaughtExceptionHandler(exceptionHandler); + th1.start(); + th1.join(); + if (exception[0] != null) { + // throw root cause. + while (exception[0].getCause() != null) { + exception[0] = exception[0].getCause(); + } + throw (Exception) exception[0]; + } + } finally { + rpcServer.stop(); + } + } + + public static class TestThread extends Thread { + private final BlockingInterface stub; + + public TestThread(BlockingInterface stub) { + this.stub = stub; + } + + @Override + public void run() { + try { + int[] messageSize = new int[] { 100, 1000, 10000 }; + for (int i = 0; i < messageSize.length; i++) { + String input = RandomStringUtils.random(messageSize[i]); + String result = + stub.echo(null, TestProtos.EchoRequestProto.newBuilder().setMessage(input).build()) + .getMessage(); + assertEquals(input, result); + } + } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestTlsRejectPlainText.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestTlsRejectPlainText.java new file mode 100644 index 000000000000..dc79ab25c492 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestTlsRejectPlainText.java @@ -0,0 +1,155 @@ +/* + * 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.security; + +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThrows; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.Security; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.exceptions.ConnectionClosedException; +import org.apache.hadoop.hbase.io.crypto.tls.KeyStoreFileType; +import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContext; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContextProvider; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcServer; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Parameterized; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; + +public abstract class AbstractTestTlsRejectPlainText { + + protected static HBaseCommonTestingUtility UTIL; + + protected static File DIR; + + protected static X509TestContextProvider PROVIDER; + + @Parameterized.Parameter(0) + public X509KeyType caKeyType; + + @Parameterized.Parameter(1) + public X509KeyType certKeyType; + + @Parameterized.Parameter(2) + public char[] keyPassword; + + private X509TestContext x509TestContext; + + protected RpcServer rpcServer; + + protected RpcClient rpcClient; + + @Parameterized.Parameters(name = "{index}: caKeyType={0}, certKeyType={1}, keyPassword={2}") + public static List data() { + List params = new ArrayList<>(); + for (X509KeyType caKeyType : X509KeyType.values()) { + for (X509KeyType certKeyType : X509KeyType.values()) { + for (char[] keyPassword : new char[][] { "".toCharArray(), "pa$$w0rd".toCharArray() }) { + params.add(new Object[] { caKeyType, certKeyType, keyPassword }); + } + } + } + return params; + } + + protected static void initialize() throws IOException { + Security.addProvider(new BouncyCastleProvider()); + DIR = + new File(UTIL.getDataTestDir(AbstractTestTlsRejectPlainText.class.getSimpleName()).toString()) + .getCanonicalFile(); + FileUtils.forceMkdir(DIR); + Configuration conf = UTIL.getConfiguration(); + conf.setClass(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, NettyRpcClient.class, + RpcClient.class); + conf.setClass(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, NettyRpcServer.class, + RpcServer.class); + conf.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_ENABLED, true); + conf.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, false); + conf.setBoolean(X509Util.HBASE_CLIENT_NETTY_TLS_ENABLED, false); + PROVIDER = new X509TestContextProvider(conf, DIR); + } + + protected static void cleanUp() { + Security.removeProvider(BouncyCastleProvider.PROVIDER_NAME); + UTIL.cleanupTestDir(); + } + + @Before + public void setUp() throws Exception { + x509TestContext = PROVIDER.get(caKeyType, certKeyType, keyPassword); + x509TestContext.setConfigurations(KeyStoreFileType.JKS, KeyStoreFileType.JKS); + Configuration conf = UTIL.getConfiguration(); + rpcServer = new NettyRpcServer(null, "testRpcServer", + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), + new InetSocketAddress("localhost", 0), conf, new FifoRpcScheduler(conf, 1), true); + rpcServer.start(); + rpcClient = new NettyRpcClient(conf); + } + + @After + public void tearDown() throws IOException { + if (rpcServer != null) { + rpcServer.stop(); + } + Closeables.close(rpcClient, true); + x509TestContext.clearConfigurations(); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_OCSP); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_CLR); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_PROTOCOL); + System.clearProperty("com.sun.net.ssl.checkRevocation"); + System.clearProperty("com.sun.security.enableCRLDP"); + Security.setProperty("ocsp.enable", Boolean.FALSE.toString()); + Security.setProperty("com.sun.security.enableCRLDP", Boolean.FALSE.toString()); + } + + protected abstract BlockingInterface createStub() throws Exception; + + @Test + public void testReject() throws Exception { + BlockingInterface stub = createStub(); + ServiceException se = assertThrows(ServiceException.class, + () -> stub.echo(null, EchoRequestProto.newBuilder().setMessage("hello world").build())); + assertThat(se.getCause(), instanceOf(ConnectionClosedException.class)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestMutualTlsClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestMutualTlsClientSide.java new file mode 100644 index 000000000000..355fb58ccf95 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestMutualTlsClientSide.java @@ -0,0 +1,82 @@ +/* + * 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.security; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.bouncycastle.operator.OperatorCreationException; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Comprehensively tests all permutations of certificate and host verification on the client side. + * Tests each permutation of that against each value of {@link CertConfig}, i.e. passing a bad cert, + * etc. See inline comments in {@link #data()} below for what the expectations are + */ +@RunWith(Parameterized.class) +@Category({ RPCTests.class, MediumTests.class }) +public class TestMutualTlsClientSide extends AbstractTestMutualTls { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMutualTlsClientSide.class); + + @Parameterized.Parameters(name = "{index}: caKeyType={0}, certKeyType={1}, keyPassword={2}, " + + "validateServerHostnames={3}, testCase={4}") + public static List data() { + List params = new ArrayList<>(); + for (X509KeyType caKeyType : X509KeyType.values()) { + for (X509KeyType certKeyType : X509KeyType.values()) { + for (String keyPassword : new String[] { "", "pa$$w0rd" }) { + // we want to run with and without validating hostnames. we encode the expected success + // criteria in the TestCase config. See below. + for (boolean validateServerHostnames : new Boolean[] { true, false }) { + // fail for non-verifiable certs or certs with bad hostnames when validateServerHostname + // is true. otherwise succeed. + params.add(new Object[] { caKeyType, certKeyType, keyPassword, false, + validateServerHostnames, CertConfig.NON_VERIFIABLE_CERT }); + params.add(new Object[] { caKeyType, certKeyType, keyPassword, !validateServerHostnames, + validateServerHostnames, CertConfig.VERIFIABLE_CERT_WITH_BAD_HOST }); + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, + validateServerHostnames, CertConfig.GOOD_CERT }); + } + } + } + } + return params; + } + + @Override + protected void initialize(Configuration serverConf, Configuration clientConf) + throws IOException, GeneralSecurityException, OperatorCreationException { + // client verifies server hostname, and injects bad certs into server conf + clientConf.setBoolean(X509Util.HBASE_CLIENT_NETTY_TLS_VERIFY_SERVER_HOSTNAME, + validateHostnames); + handleCertConfig(serverConf); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestMutualTlsServerSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestMutualTlsServerSide.java new file mode 100644 index 000000000000..b15299f485d7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestMutualTlsServerSide.java @@ -0,0 +1,116 @@ +/* + * 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.security; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.bouncycastle.operator.OperatorCreationException; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Comprehensively tests all permutations of ClientAuth modes and host verification + * enabled/disabled. Tests each permutation of that against each relevant value of + * {@link CertConfig}, i.e. passing no cert, a bad cert, etc. See inline comments in {@link #data()} + * below for what the expectations are + */ +@RunWith(Parameterized.class) +@Category({ RPCTests.class, MediumTests.class }) +public class TestMutualTlsServerSide extends AbstractTestMutualTls { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMutualTlsServerSide.class); + @Parameterized.Parameter(6) + public X509Util.ClientAuth clientAuthMode; + + @Parameterized.Parameters(name = "{index}: caKeyType={0}, certKeyType={1}, keyPassword={2}, " + + "validateClientHostnames={3}, testCase={4}, clientAuthMode={5}") + public static List data() { + List params = new ArrayList<>(); + for (X509KeyType caKeyType : X509KeyType.values()) { + for (X509KeyType certKeyType : X509KeyType.values()) { + for (String keyPassword : new String[] { "", "pa$$w0rd" }) { + // we want to run with and without validating hostnames. we encode the expected success + // criteria + // in the TestCase config. See below. + for (boolean validateClientHostnames : new Boolean[] { true, false }) { + // ClientAuth.NONE should succeed in all cases, because it never requests the + // certificate for verification + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, + validateClientHostnames, CertConfig.NO_CLIENT_CERT, X509Util.ClientAuth.NONE }); + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, + validateClientHostnames, CertConfig.NON_VERIFIABLE_CERT, X509Util.ClientAuth.NONE }); + params.add( + new Object[] { caKeyType, certKeyType, keyPassword, true, validateClientHostnames, + CertConfig.VERIFIABLE_CERT_WITH_BAD_HOST, X509Util.ClientAuth.NONE }); + + // ClientAuth.WANT should succeed if no cert, but if the cert is provided it is + // validated. So should fail on bad cert or good cert with bad host when host + // verification is enabled + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, + validateClientHostnames, CertConfig.NO_CLIENT_CERT, X509Util.ClientAuth.WANT }); + params.add(new Object[] { caKeyType, certKeyType, keyPassword, false, + validateClientHostnames, CertConfig.NON_VERIFIABLE_CERT, X509Util.ClientAuth.WANT }); + params.add(new Object[] { caKeyType, certKeyType, keyPassword, !validateClientHostnames, + validateClientHostnames, CertConfig.VERIFIABLE_CERT_WITH_BAD_HOST, + X509Util.ClientAuth.WANT }); + + // ClientAuth.NEED is most restrictive, failing in all cases except "good cert/bad host" + // when host verification is disabled + params.add(new Object[] { caKeyType, certKeyType, keyPassword, false, + validateClientHostnames, CertConfig.NO_CLIENT_CERT, X509Util.ClientAuth.NEED }); + params.add(new Object[] { caKeyType, certKeyType, keyPassword, false, + validateClientHostnames, CertConfig.NON_VERIFIABLE_CERT, X509Util.ClientAuth.NEED }); + params.add(new Object[] { caKeyType, certKeyType, keyPassword, !validateClientHostnames, + validateClientHostnames, CertConfig.VERIFIABLE_CERT_WITH_BAD_HOST, + X509Util.ClientAuth.NEED }); + + // additionally ensure that all modes succeed when a good cert is presented + for (X509Util.ClientAuth mode : X509Util.ClientAuth.values()) { + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, + validateClientHostnames, CertConfig.GOOD_CERT, mode }); + } + } + } + } + } + return params; + } + + @Override + protected void initialize(Configuration serverConf, Configuration clientConf) + throws IOException, GeneralSecurityException, OperatorCreationException { + // server enables client auth mode and verifies client host names + // inject bad certs into client side + serverConf.set(X509Util.HBASE_SERVER_NETTY_TLS_CLIENT_AUTH_MODE, clientAuthMode.name()); + serverConf.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_VERIFY_CLIENT_HOSTNAME, + validateHostnames); + handleCertConfig(clientConf); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyIPCSslFailure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyIPCSslFailure.java new file mode 100644 index 000000000000..0809681de12d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyIPCSslFailure.java @@ -0,0 +1,96 @@ +/* + * 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.security; + +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; + +import java.io.IOException; +import java.net.InetSocketAddress; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.exceptions.SSLContextException; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper; +import org.apache.hadoop.hbase.ipc.NettyRpcServer; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos; + +@Category({ RPCTests.class, MediumTests.class }) +public class TestNettyIPCSslFailure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNettyIPCSslFailure.class); + + private static final Configuration CONF = HBaseConfiguration.create(); + + private NioEventLoopGroup group; + + private NettyRpcServer server; + + private NettyRpcClient client; + + private TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub; + + @Before + public void setUp() throws IOException, SSLContextException { + CONF.set(X509Util.HBASE_SERVER_NETTY_TLS_ENABLED, "true"); + CONF.unset(X509Util.TLS_CONFIG_KEYSTORE_LOCATION); + group = new NioEventLoopGroup(); + server = new NettyRpcServer(null, getClass().getSimpleName(), + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), + new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1), true); + NettyRpcClientConfigHelper.setEventLoopConfig(CONF, group, NioSocketChannel.class); + client = new NettyRpcClient(CONF); + server.start(); + stub = TestProtobufRpcServiceImpl.newBlockingStub(client, server.getListenerAddress()); + } + + @After + public void tearDown() throws Exception { + Closeables.close(client, true); + server.stop(); + group.shutdownGracefully().sync(); + } + + @Test(expected = ServiceException.class) + public void testInitSslThrowsException() throws ServiceException { + stub.echo(null, TestProtos.EchoRequestProto.newBuilder().setMessage("test").build()) + .getMessage(); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTLSIPCFileWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTLSIPCFileWatcher.java new file mode 100644 index 000000000000..34c812a3bb90 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTLSIPCFileWatcher.java @@ -0,0 +1,234 @@ +/* + * 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.security; + +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.GeneralSecurityException; +import java.security.Security; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.io.crypto.tls.KeyStoreFileType; +import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContext; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContextProvider; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.ipc.AbstractRpcClient; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcServer; +import org.apache.hadoop.hbase.ipc.RpcScheduler; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.net.Address; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.operator.OperatorCreationException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos; + +@RunWith(Parameterized.class) +@Category({ RPCTests.class, MediumTests.class }) +public class TestNettyTLSIPCFileWatcher { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNettyTLSIPCFileWatcher.class); + + private static final Configuration CONF = HBaseConfiguration.create(); + private static final HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility(CONF); + private static HRegionServer SERVER; + private static X509TestContextProvider PROVIDER; + private static NettyEventLoopGroupConfig EVENT_LOOP_GROUP_CONFIG; + + private X509TestContext x509TestContext; + + @Parameterized.Parameter(0) + public X509KeyType keyType; + + @Parameterized.Parameter(1) + public KeyStoreFileType storeFileType; + + @Parameterized.Parameters(name = "{index}: keyType={0}, storeFileType={1}") + public static List data() { + List params = new ArrayList<>(); + for (X509KeyType caKeyType : X509KeyType.values()) { + for (KeyStoreFileType ks : KeyStoreFileType.values()) { + params.add(new Object[] { caKeyType, ks }); + } + } + return params; + } + + @BeforeClass + public static void setUpBeforeClass() throws IOException { + Security.addProvider(new BouncyCastleProvider()); + File dir = new File(UTIL.getDataTestDir(TestNettyTlsIPC.class.getSimpleName()).toString()) + .getCanonicalFile(); + FileUtils.forceMkdir(dir); + // server must enable tls + CONF.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_ENABLED, true); + PROVIDER = new X509TestContextProvider(CONF, dir); + EVENT_LOOP_GROUP_CONFIG = + new NettyEventLoopGroupConfig(CONF, TestNettyTLSIPCFileWatcher.class.getSimpleName()); + SERVER = mock(HRegionServer.class); + when(SERVER.getEventLoopGroupConfig()).thenReturn(EVENT_LOOP_GROUP_CONFIG); + } + + @AfterClass + public static void tearDownAfterClass() throws InterruptedException { + Security.removeProvider(BouncyCastleProvider.PROVIDER_NAME); + EVENT_LOOP_GROUP_CONFIG.group().shutdownGracefully().sync(); + UTIL.cleanupTestDir(); + } + + @Before + public void setUp() throws IOException { + x509TestContext = PROVIDER.get(keyType, keyType, "keyPa$$word".toCharArray()); + x509TestContext.setConfigurations(storeFileType, storeFileType); + CONF.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, false); + CONF.setBoolean(X509Util.HBASE_CLIENT_NETTY_TLS_ENABLED, true); + CONF.setBoolean(X509Util.TLS_CERT_RELOAD, true); + } + + @After + public void tearDown() { + x509TestContext.clearConfigurations(); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_OCSP); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_CLR); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_PROTOCOL); + System.clearProperty("com.sun.net.ssl.checkRevocation"); + System.clearProperty("com.sun.security.enableCRLDP"); + Security.setProperty("ocsp.enable", Boolean.FALSE.toString()); + Security.setProperty("com.sun.security.enableCRLDP", Boolean.FALSE.toString()); + } + + @Test + public void testReplaceServerKeystore() + throws IOException, ServiceException, GeneralSecurityException, OperatorCreationException { + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), + new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); + + try { + rpcServer.start(); + + try (AbstractRpcClient client = new NettyRpcClient(clientConf)) { + TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub = + newBlockingStub(client, rpcServer.getListenerAddress()); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + String message = "hello"; + assertEquals(message, + stub.echo(pcrc, TestProtos.EchoRequestProto.newBuilder().setMessage(message).build()) + .getMessage()); + assertNull(pcrc.cellScanner()); + } + + // Replace keystore + x509TestContext.regenerateStores(keyType, keyType, storeFileType, storeFileType); + + try (AbstractRpcClient client = new NettyRpcClient(clientConf)) { + TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub = + newBlockingStub(client, rpcServer.getListenerAddress()); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + String message = "hello"; + assertEquals(message, + stub.echo(pcrc, TestProtos.EchoRequestProto.newBuilder().setMessage(message).build()) + .getMessage()); + assertNull(pcrc.cellScanner()); + } + + } finally { + rpcServer.stop(); + } + } + + @Test + public void testReplaceClientAndServerKeystore() + throws GeneralSecurityException, IOException, OperatorCreationException, ServiceException { + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), + new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); + + try { + rpcServer.start(); + + try (AbstractRpcClient client = new NettyRpcClient(clientConf)) { + TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub = + newBlockingStub(client, rpcServer.getListenerAddress()); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + String message = "hello"; + assertEquals(message, + stub.echo(pcrc, TestProtos.EchoRequestProto.newBuilder().setMessage(message).build()) + .getMessage()); + assertNull(pcrc.cellScanner()); + + // Replace keystore and cancel client connections + x509TestContext.regenerateStores(keyType, keyType, storeFileType, storeFileType); + client.cancelConnections( + ServerName.valueOf(Address.fromSocketAddress(rpcServer.getListenerAddress()), 0L)); + + assertEquals(message, + stub.echo(pcrc, TestProtos.EchoRequestProto.newBuilder().setMessage(message).build()) + .getMessage()); + assertNull(pcrc.cellScanner()); + } + } finally { + rpcServer.stop(); + } + } + + private RpcServer createRpcServer(String name, + List services, InetSocketAddress bindAddress, + Configuration conf, RpcScheduler scheduler) throws IOException { + return new NettyRpcServer(SERVER, name, services, bindAddress, conf, scheduler, true); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPC.java new file mode 100644 index 000000000000..65460ab3eaf2 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPC.java @@ -0,0 +1,202 @@ +/* + * 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.security; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.Security; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.io.crypto.tls.KeyStoreFileType; +import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContext; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContextProvider; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.ipc.AbstractRpcClient; +import org.apache.hadoop.hbase.ipc.AbstractTestIPC; +import org.apache.hadoop.hbase.ipc.FailingNettyRpcServer; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper; +import org.apache.hadoop.hbase.ipc.NettyRpcServer; +import org.apache.hadoop.hbase.ipc.RpcScheduler; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig; +import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +@Category({ RPCTests.class, MediumTests.class }) +public class TestNettyTlsIPC extends AbstractTestIPC { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNettyTlsIPC.class); + + private static final HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility(CONF); + + private static X509TestContextProvider PROVIDER; + + private static NettyEventLoopGroupConfig EVENT_LOOP_GROUP_CONFIG; + + private static HRegionServer SERVER; + + @Parameterized.Parameter(0) + public X509KeyType caKeyType; + + @Parameterized.Parameter(1) + public X509KeyType certKeyType; + + @Parameterized.Parameter(2) + public char[] keyPassword; + + @Parameterized.Parameter(3) + public boolean acceptPlainText; + + @Parameterized.Parameter(4) + public boolean clientTlsEnabled; + + private X509TestContext x509TestContext; + + @Parameterized.Parameters( + name = "{index}: caKeyType={0}, certKeyType={1}, keyPassword={2}, acceptPlainText={3}," + + " clientTlsEnabled={4}") + public static List data() { + List params = new ArrayList<>(); + for (X509KeyType caKeyType : X509KeyType.values()) { + for (X509KeyType certKeyType : X509KeyType.values()) { + for (char[] keyPassword : new char[][] { "".toCharArray(), "pa$$w0rd".toCharArray() }) { + // do not accept plain text + params.add(new Object[] { caKeyType, certKeyType, keyPassword, false, true }); + // support plain text and client enables tls + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, true }); + // support plain text and client disables tls + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, false }); + } + } + } + return params; + } + + @BeforeClass + public static void setUpBeforeClass() throws IOException { + Security.addProvider(new BouncyCastleProvider()); + File dir = new File(UTIL.getDataTestDir(TestNettyTlsIPC.class.getSimpleName()).toString()) + .getCanonicalFile(); + FileUtils.forceMkdir(dir); + // server must enable tls + CONF.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_ENABLED, true); + PROVIDER = new X509TestContextProvider(CONF, dir); + EVENT_LOOP_GROUP_CONFIG = + new NettyEventLoopGroupConfig(CONF, TestNettyTlsIPC.class.getSimpleName()); + NettyRpcClientConfigHelper.setEventLoopConfig(CONF, EVENT_LOOP_GROUP_CONFIG.group(), + EVENT_LOOP_GROUP_CONFIG.clientChannelClass()); + NettyAsyncFSWALConfigHelper.setEventLoopConfig(CONF, EVENT_LOOP_GROUP_CONFIG.group(), + EVENT_LOOP_GROUP_CONFIG.clientChannelClass()); + SERVER = mock(HRegionServer.class); + when(SERVER.getEventLoopGroupConfig()).thenReturn(EVENT_LOOP_GROUP_CONFIG); + } + + @AfterClass + public static void tearDownAfterClass() throws InterruptedException { + Security.removeProvider(BouncyCastleProvider.PROVIDER_NAME); + EVENT_LOOP_GROUP_CONFIG.group().shutdownGracefully().sync(); + UTIL.cleanupTestDir(); + } + + @Before + public void setUp() throws IOException { + x509TestContext = PROVIDER.get(caKeyType, certKeyType, keyPassword); + x509TestContext.setConfigurations(KeyStoreFileType.JKS, KeyStoreFileType.JKS); + CONF.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, acceptPlainText); + CONF.setBoolean(X509Util.HBASE_CLIENT_NETTY_TLS_ENABLED, clientTlsEnabled); + } + + @After + public void tearDown() { + x509TestContext.clearConfigurations(); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_OCSP); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_CLR); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_PROTOCOL); + System.clearProperty("com.sun.net.ssl.checkRevocation"); + System.clearProperty("com.sun.security.enableCRLDP"); + Security.setProperty("ocsp.enable", Boolean.FALSE.toString()); + Security.setProperty("com.sun.security.enableCRLDP", Boolean.FALSE.toString()); + } + + @Override + protected RpcServer createRpcServer(String name, List services, + InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { + return new NettyRpcServer(SERVER, name, services, bindAddress, conf, scheduler, true); + } + + @Override + protected AbstractRpcClient createRpcClientNoCodec(Configuration conf) { + return new NettyRpcClient(conf) { + + @Override + protected Codec getCodec() { + return null; + } + }; + } + + @Override + protected AbstractRpcClient createRpcClient(Configuration conf) { + return new NettyRpcClient(conf); + } + + @Override + protected AbstractRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf) + throws IOException { + return new NettyRpcClient(conf) { + + @Override + protected boolean isTcpNoDelay() { + throw new RuntimeException("Injected fault"); + } + }; + } + + @Override + protected RpcServer createTestFailingRpcServer(String name, + List services, InetSocketAddress bindAddress, Configuration conf, + RpcScheduler scheduler) throws IOException { + return new FailingNettyRpcServer(SERVER, name, services, bindAddress, conf, scheduler); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPCRejectPlainText.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPCRejectPlainText.java new file mode 100644 index 000000000000..fcb14aa85e8b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPCRejectPlainText.java @@ -0,0 +1,58 @@ +/* + * 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.security; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; + +@RunWith(Parameterized.class) +@Category({ RPCTests.class, MediumTests.class }) +public class TestNettyTlsIPCRejectPlainText extends AbstractTestTlsRejectPlainText { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNettyTlsIPCRejectPlainText.class); + + @BeforeClass + public static void setUpBeforeClass() throws IOException { + UTIL = new HBaseCommonTestingUtility(); + initialize(); + } + + @AfterClass + public static void tearDownAfterClass() { + cleanUp(); + } + + @Override + protected BlockingInterface createStub() throws Exception { + return TestProtobufRpcServiceImpl.newBlockingStub(rpcClient, rpcServer.getListenerAddress()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPC.java new file mode 100644 index 000000000000..1477e8aa0fca --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPC.java @@ -0,0 +1,143 @@ +/* + * 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.security; + +import java.io.File; +import java.security.Security; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.io.crypto.tls.KeyStoreFileType; +import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContext; +import org.apache.hadoop.hbase.io.crypto.tls.X509TestContextProvider; +import org.apache.hadoop.hbase.io.crypto.tls.X509Util; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcServer; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +@Category({ SecurityTests.class, LargeTests.class }) +public class TestSaslTlsIPC extends AbstractTestSecureIPC { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSaslTlsIPC.class); + + private static X509TestContextProvider PROVIDER; + + @Parameterized.Parameter(0) + public X509KeyType caKeyType; + + @Parameterized.Parameter(1) + public X509KeyType certKeyType; + + @Parameterized.Parameter(2) + public char[] keyPassword; + + @Parameterized.Parameter(3) + public boolean acceptPlainText; + + @Parameterized.Parameter(4) + public boolean clientTlsEnabled; + + private X509TestContext x509TestContext; + + @Parameterized.Parameters( + name = "{index}: caKeyType={0}, certKeyType={1}, keyPassword={2}, acceptPlainText={3}," + + " clientTlsEnabled={4}") + public static List data() { + List params = new ArrayList<>(); + for (X509KeyType caKeyType : X509KeyType.values()) { + for (X509KeyType certKeyType : X509KeyType.values()) { + for (char[] keyPassword : new char[][] { "".toCharArray(), "pa$$w0rd".toCharArray() }) { + // do not accept plain text + params.add(new Object[] { caKeyType, certKeyType, keyPassword, false, true }); + // support plain text and client enables tls + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, true }); + // support plain text and client disables tls + params.add(new Object[] { caKeyType, certKeyType, keyPassword, true, false }); + } + } + } + return params; + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + Security.addProvider(new BouncyCastleProvider()); + File dir = new File(TEST_UTIL.getDataTestDir(TestNettyTlsIPC.class.getSimpleName()).toString()) + .getCanonicalFile(); + FileUtils.forceMkdir(dir); + initKDCAndConf(); + Configuration conf = TEST_UTIL.getConfiguration(); + // server must enable tls + conf.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_ENABLED, true); + // only netty support tls + conf.setClass(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, NettyRpcClient.class, + RpcClient.class); + conf.setClass(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, NettyRpcServer.class, + RpcServer.class); + PROVIDER = new X509TestContextProvider(conf, dir); + } + + @AfterClass + public static void tearDownAfterClass() throws InterruptedException { + stopKDC(); + Security.removeProvider(BouncyCastleProvider.PROVIDER_NAME); + TEST_UTIL.cleanupTestDir(); + } + + @Before + public void setUp() throws Exception { + x509TestContext = PROVIDER.get(caKeyType, certKeyType, keyPassword); + x509TestContext.setConfigurations(KeyStoreFileType.JKS, KeyStoreFileType.JKS); + Configuration conf = TEST_UTIL.getConfiguration(); + conf.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, acceptPlainText); + conf.setBoolean(X509Util.HBASE_CLIENT_NETTY_TLS_ENABLED, clientTlsEnabled); + setUpPrincipalAndConf(); + } + + @After + public void tearDown() { + x509TestContext.clearConfigurations(); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_OCSP); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_CLR); + x509TestContext.getConf().unset(X509Util.TLS_CONFIG_PROTOCOL); + System.clearProperty("com.sun.net.ssl.checkRevocation"); + System.clearProperty("com.sun.security.enableCRLDP"); + Security.setProperty("ocsp.enable", Boolean.FALSE.toString()); + Security.setProperty("com.sun.security.enableCRLDP", Boolean.FALSE.toString()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPCRejectPlainText.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPCRejectPlainText.java new file mode 100644 index 000000000000..0b54e9a65125 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPCRejectPlainText.java @@ -0,0 +1,87 @@ +/* + * 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.security; + +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.loginKerberosPrincipal; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.setSecuredConfiguration; + +import java.io.File; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.minikdc.MiniKdc; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.mockito.Mockito; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; + +@RunWith(Parameterized.class) +@Category({ SecurityTests.class, MediumTests.class }) +public class TestSaslTlsIPCRejectPlainText extends AbstractTestTlsRejectPlainText { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSaslTlsIPCRejectPlainText.class); + + private static File KEYTAB_FILE; + + private static MiniKdc KDC; + private static String HOST = "localhost"; + private static String PRINCIPAL; + private static UserGroupInformation UGI; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + HBaseTestingUtility util = new HBaseTestingUtility(); + UTIL = util; + initialize(); + KEYTAB_FILE = new File(util.getDataTestDir("keytab").toUri().getPath()); + KDC = util.setupMiniKdc(KEYTAB_FILE); + PRINCIPAL = "hbase/" + HOST; + KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL); + HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm()); + UGI = loginKerberosPrincipal(KEYTAB_FILE.getCanonicalPath(), PRINCIPAL); + setSecuredConfiguration(util.getConfiguration()); + SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class); + Mockito.when(securityInfoMock.getServerPrincipal()) + .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL); + SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock); + } + + @AfterClass + public static void tearDownAfterClass() { + if (KDC != null) { + KDC.stop(); + } + cleanUp(); + } + + @Override + protected BlockingInterface createStub() throws Exception { + return TestProtobufRpcServiceImpl.newBlockingStub(rpcClient, rpcServer.getListenerAddress(), + User.create(UGI)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java index 67d9803bf298..991c3a936cbd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java @@ -17,101 +17,37 @@ */ package org.apache.hadoop.hbase.security; -import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; -import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; -import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting; -import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting; -import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.setSecuredConfiguration; -import static org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProviders.SELECTOR_KEY; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.fail; - -import java.io.File; -import java.io.IOException; -import java.lang.reflect.Field; -import java.net.InetAddress; -import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Map; -import javax.security.sasl.SaslClient; -import javax.security.sasl.SaslException; -import org.apache.commons.lang3.RandomStringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ipc.BlockingRpcClient; -import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; import org.apache.hadoop.hbase.ipc.NettyRpcClient; import org.apache.hadoop.hbase.ipc.NettyRpcServer; -import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.RpcServerFactory; -import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.ipc.SimpleRpcServer; -import org.apache.hadoop.hbase.security.provider.AuthenticationProviderSelector; -import org.apache.hadoop.hbase.security.provider.BuiltInProviderSelector; -import org.apache.hadoop.hbase.security.provider.SaslAuthMethod; -import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProvider; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.minikdc.MiniKdc; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; -import org.mockito.Mockito; - -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; - -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation; @RunWith(Parameterized.class) @Category({ SecurityTests.class, LargeTests.class }) -public class TestSecureIPC { +public class TestSecureIPC extends AbstractTestSecureIPC { @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestSecureIPC.class); - private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - private static final File KEYTAB_FILE = - new File(TEST_UTIL.getDataTestDir("keytab").toUri().getPath()); - - private static MiniKdc KDC; - private static String HOST = "localhost"; - private static String PRINCIPAL; - - private String krbKeytab; - private String krbPrincipal; - private UserGroupInformation ugi; - private Configuration clientConf; - private Configuration serverConf; - @Parameters(name = "{index}: rpcClientImpl={0}, rpcServerImpl={1}") public static Collection parameters() { List params = new ArrayList<>(); @@ -135,317 +71,19 @@ public static Collection parameters() { @BeforeClass public static void setUp() throws Exception { - KDC = TEST_UTIL.setupMiniKdc(KEYTAB_FILE); - PRINCIPAL = "hbase/" + HOST; - KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL); - HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm()); - // set a smaller timeout and retry to speed up tests - TEST_UTIL.getConfiguration().setInt(RpcClient.SOCKET_TIMEOUT_READ, 2000); - TEST_UTIL.getConfiguration().setInt("hbase.security.relogin.maxretries", 1); + initKDCAndConf(); } @AfterClass - public static void tearDown() throws IOException { - if (KDC != null) { - KDC.stop(); - } + public static void tearDown() throws Exception { + stopKDC(); TEST_UTIL.cleanupTestDir(); } @Before public void setUpTest() throws Exception { - krbKeytab = getKeytabFileForTesting(); - krbPrincipal = getPrincipalForTesting(); - ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal); - clientConf = new Configuration(TEST_UTIL.getConfiguration()); - setSecuredConfiguration(clientConf); + setUpPrincipalAndConf(); clientConf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, rpcClientImpl); - serverConf = new Configuration(TEST_UTIL.getConfiguration()); - setSecuredConfiguration(serverConf); serverConf.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, rpcServerImpl); } - - @Test - public void testRpcCallWithEnabledKerberosSaslAuth() throws Exception { - UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); - - // check that the login user is okay: - assertSame(ugi2, ugi); - assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - assertEquals(krbPrincipal, ugi.getUserName()); - - callRpcService(User.create(ugi2)); - } - - @Test - public void testRpcCallWithEnabledKerberosSaslAuth_CanonicalHostname() throws Exception { - UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); - - // check that the login user is okay: - assertSame(ugi2, ugi); - assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - assertEquals(krbPrincipal, ugi.getUserName()); - - enableCanonicalHostnameTesting(clientConf, "localhost"); - clientConf.setBoolean( - SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS, false); - clientConf.set(HBaseKerberosUtils.KRB_PRINCIPAL, "hbase/_HOST@" + KDC.getRealm()); - - callRpcService(User.create(ugi2)); - } - - @Test - public void testRpcCallWithEnabledKerberosSaslAuth_NoCanonicalHostname() throws Exception { - UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); - - // check that the login user is okay: - assertSame(ugi2, ugi); - assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - assertEquals(krbPrincipal, ugi.getUserName()); - - enableCanonicalHostnameTesting(clientConf, "127.0.0.1"); - clientConf - .setBoolean(SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS, true); - clientConf.set(HBaseKerberosUtils.KRB_PRINCIPAL, "hbase/_HOST@" + KDC.getRealm()); - - callRpcService(User.create(ugi2)); - } - - private static void enableCanonicalHostnameTesting(Configuration conf, String canonicalHostname) { - conf.setClass(SELECTOR_KEY, CanonicalHostnameTestingAuthenticationProviderSelector.class, - AuthenticationProviderSelector.class); - conf.set(CanonicalHostnameTestingAuthenticationProviderSelector.CANONICAL_HOST_NAME_KEY, - canonicalHostname); - } - - public static class CanonicalHostnameTestingAuthenticationProviderSelector - extends BuiltInProviderSelector { - private static final String CANONICAL_HOST_NAME_KEY = - "CanonicalHostnameTestingAuthenticationProviderSelector.canonicalHostName"; - - @Override - public Pair> - selectProvider(String clusterId, User user) { - final Pair> pair = - super.selectProvider(clusterId, user); - pair.setFirst(createCanonicalHostNameTestingProvider(pair.getFirst())); - return pair; - } - - SaslClientAuthenticationProvider - createCanonicalHostNameTestingProvider(SaslClientAuthenticationProvider delegate) { - return new SaslClientAuthenticationProvider() { - @Override - public SaslClient createClient(Configuration conf, InetAddress serverAddr, - SecurityInfo securityInfo, Token token, - boolean fallbackAllowed, Map saslProps) throws IOException { - final String s = conf.get(CANONICAL_HOST_NAME_KEY); - if (s != null) { - try { - final Field canonicalHostName = - InetAddress.class.getDeclaredField("canonicalHostName"); - canonicalHostName.setAccessible(true); - canonicalHostName.set(serverAddr, s); - } catch (NoSuchFieldException | IllegalAccessException e) { - throw new RuntimeException(e); - } - } - - return delegate.createClient(conf, serverAddr, securityInfo, token, fallbackAllowed, - saslProps); - } - - @Override - public UserInformation getUserInfo(User user) { - return delegate.getUserInfo(user); - } - - @Override - public UserGroupInformation getRealUser(User ugi) { - return delegate.getRealUser(ugi); - } - - @Override - public boolean canRetry() { - return delegate.canRetry(); - } - - @Override - public void relogin() throws IOException { - delegate.relogin(); - } - - @Override - public SaslAuthMethod getSaslAuthMethod() { - return delegate.getSaslAuthMethod(); - } - - @Override - public String getTokenKind() { - return delegate.getTokenKind(); - } - }; - } - } - - @Test - public void testRpcFallbackToSimpleAuth() throws Exception { - String clientUsername = "testuser"; - UserGroupInformation clientUgi = - UserGroupInformation.createUserForTesting(clientUsername, new String[] { clientUsername }); - - // check that the client user is insecure - assertNotSame(ugi, clientUgi); - assertEquals(AuthenticationMethod.SIMPLE, clientUgi.getAuthenticationMethod()); - assertEquals(clientUsername, clientUgi.getUserName()); - - clientConf.set(User.HBASE_SECURITY_CONF_KEY, "simple"); - serverConf.setBoolean(RpcServer.FALLBACK_TO_INSECURE_CLIENT_AUTH, true); - callRpcService(User.create(clientUgi)); - } - - private void setRpcProtection(String clientProtection, String serverProtection) { - clientConf.set("hbase.rpc.protection", clientProtection); - serverConf.set("hbase.rpc.protection", serverProtection); - } - - /** - * Test various combinations of Server and Client qops. - */ - @Test - public void testSaslWithCommonQop() throws Exception { - setRpcProtection("privacy,authentication", "authentication"); - callRpcService(User.create(ugi)); - - setRpcProtection("authentication", "privacy,authentication"); - callRpcService(User.create(ugi)); - - setRpcProtection("integrity,authentication", "privacy,authentication"); - callRpcService(User.create(ugi)); - - setRpcProtection("integrity,authentication", "integrity,authentication"); - callRpcService(User.create(ugi)); - - setRpcProtection("privacy,authentication", "privacy,authentication"); - callRpcService(User.create(ugi)); - } - - @Test - public void testSaslNoCommonQop() throws Exception { - setRpcProtection("integrity", "privacy"); - SaslException se = assertThrows(SaslException.class, () -> callRpcService(User.create(ugi))); - assertEquals("No common protection layer between client and server", se.getMessage()); - } - - /** - * Test sasl encryption with Crypto AES. - */ - @Test - public void testSaslWithCryptoAES() throws Exception { - setRpcProtection("privacy", "privacy"); - setCryptoAES("true", "true"); - callRpcService(User.create(ugi)); - } - - /** - * Test various combinations of Server and Client configuration for Crypto AES. - */ - @Test - public void testDifferentConfWithCryptoAES() throws Exception { - setRpcProtection("privacy", "privacy"); - - setCryptoAES("false", "true"); - callRpcService(User.create(ugi)); - - setCryptoAES("true", "false"); - try { - callRpcService(User.create(ugi)); - fail("The exception should be thrown out for the rpc timeout."); - } catch (Exception e) { - // ignore the expected exception - } - } - - void setCryptoAES(String clientCryptoAES, String serverCryptoAES) { - clientConf.set("hbase.rpc.crypto.encryption.aes.enabled", clientCryptoAES); - serverConf.set("hbase.rpc.crypto.encryption.aes.enabled", serverCryptoAES); - } - - private UserGroupInformation loginKerberosPrincipal(String krbKeytab, String krbPrincipal) - throws Exception { - Configuration cnf = new Configuration(); - cnf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - UserGroupInformation.setConfiguration(cnf); - UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab); - return UserGroupInformation.getLoginUser(); - } - - /** - * Sets up a RPC Server and a Client. Does a RPC checks the result. If an exception is thrown from - * the stub, this function will throw root cause of that exception. - */ - private void callRpcService(User clientUser) throws Exception { - SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class); - Mockito.when(securityInfoMock.getServerPrincipal()) - .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL); - SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock); - - InetSocketAddress isa = new InetSocketAddress(HOST, 0); - - RpcServerInterface rpcServer = RpcServerFactory.createRpcServer(null, "AbstractTestSecureIPC", - Lists - .newArrayList(new RpcServer.BlockingServiceAndInterface((BlockingService) SERVICE, null)), - isa, serverConf, new FifoRpcScheduler(serverConf, 1)); - rpcServer.start(); - try (RpcClient rpcClient = - RpcClientFactory.createClient(clientConf, HConstants.DEFAULT_CLUSTER_ID.toString())) { - BlockingInterface stub = - newBlockingStub(rpcClient, rpcServer.getListenerAddress(), clientUser); - TestThread th1 = new TestThread(stub); - final Throwable exception[] = new Throwable[1]; - Collections.synchronizedList(new ArrayList()); - Thread.UncaughtExceptionHandler exceptionHandler = new Thread.UncaughtExceptionHandler() { - @Override - public void uncaughtException(Thread th, Throwable ex) { - exception[0] = ex; - } - }; - th1.setUncaughtExceptionHandler(exceptionHandler); - th1.start(); - th1.join(); - if (exception[0] != null) { - // throw root cause. - while (exception[0].getCause() != null) { - exception[0] = exception[0].getCause(); - } - throw (Exception) exception[0]; - } - } finally { - rpcServer.stop(); - } - } - - public static class TestThread extends Thread { - private final BlockingInterface stub; - - public TestThread(BlockingInterface stub) { - this.stub = stub; - } - - @Override - public void run() { - try { - int[] messageSize = new int[] { 100, 1000, 10000 }; - for (int i = 0; i < messageSize.length; i++) { - String input = RandomStringUtils.random(messageSize[i]); - String result = - stub.echo(null, TestProtos.EchoRequestProto.newBuilder().setMessage(input).build()) - .getMessage(); - assertEquals(input, result); - } - } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException e) { - throw new RuntimeException(e); - } - } - } } diff --git a/pom.xml b/pom.xml index c1a8196ac9d4..a974dfa5e0cb 100644 --- a/pom.xml +++ b/pom.xml @@ -576,8 +576,8 @@ 4.5.13 4.4.13 3.2.6 - 2.14.0 - 2.14.0 + 2.14.1 + 2.14.1 2.3.1 3.1.0 2.1.1 @@ -647,7 +647,7 @@ 1.1.8.4 1.9 1.5.0-4 - 4.1.3 + 4.1.4 @@ -1380,6 +1380,12 @@ ${skyscreamer.version} test + + org.bouncycastle + bcpkix-jdk15on + ${bouncycastle.version} + test + org.apache.kerby kerb-core