diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java index 5ae55087652..15eb10cb1d7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java @@ -60,12 +60,14 @@ import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; import org.apache.hadoop.hbase.security.HBaseSaslRpcClient; +import org.apache.hadoop.hbase.security.SaslUtil; import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ExceptionUtil; @@ -111,6 +113,8 @@ class BlockingRpcConnection extends RpcConnection implements Runnable { private byte[] connectionHeaderWithLength; + private boolean waitingConnectionHeaderResponse = false; + /** * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt), it gets into a * java issue: an interruption during a write closes the socket/channel. A way to avoid this is to @@ -349,7 +353,8 @@ class BlockingRpcConnection extends RpcConnection implements Runnable { throws IOException { saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, this.rpcClient.fallbackAllowed, this.rpcClient.conf.get("hbase.rpc.protection", - QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT))); + QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)), + this.rpcClient.conf.getBoolean(CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT)); return saslRpcClient.saslConnect(in2, out2); } @@ -462,8 +467,8 @@ class BlockingRpcConnection extends RpcConnection implements Runnable { } if (continueSasl) { // Sasl connect is successful. Let's set up Sasl i/o streams. - inStream = saslRpcClient.getInputStream(inStream); - outStream = saslRpcClient.getOutputStream(outStream); + inStream = saslRpcClient.getInputStream(); + outStream = saslRpcClient.getOutputStream(); } else { // fall back to simple auth because server told us so. // do not change authMethod and useSasl here, we should start from secure when @@ -474,6 +479,9 @@ class BlockingRpcConnection extends RpcConnection implements Runnable { this.out = new DataOutputStream(new BufferedOutputStream(outStream)); // Now write out the connection header writeConnectionHeader(); + // process the response from server for connection header if necessary + processResponseForConnectionHeader(); + break; } } catch (Throwable t) { @@ -511,10 +519,60 @@ class BlockingRpcConnection extends RpcConnection implements Runnable { * Write the connection header. */ private void writeConnectionHeader() throws IOException { + boolean isCryptoAesEnable = false; + // check if Crypto AES is enabled + if (saslRpcClient != null) { + boolean saslEncryptionEnabled = SaslUtil.QualityOfProtection.PRIVACY. + getSaslQop().equalsIgnoreCase(saslRpcClient.getSaslQOP()); + isCryptoAesEnable = saslEncryptionEnabled && conf.getBoolean( + CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT); + } + + // if Crypto AES is enabled, set transformation and negotiate with server + if (isCryptoAesEnable) { + waitingConnectionHeaderResponse = true; + } this.out.write(connectionHeaderWithLength); this.out.flush(); } + private void processResponseForConnectionHeader() throws IOException { + // if no response excepted, return + if (!waitingConnectionHeaderResponse) return; + try { + // read the ConnectionHeaderResponse from server + int len = this.in.readInt(); + byte[] buff = new byte[len]; + int readSize = this.in.read(buff); + if (LOG.isDebugEnabled()) { + LOG.debug("Length of response for connection header:" + readSize); + } + + RPCProtos.ConnectionHeaderResponse connectionHeaderResponse = + RPCProtos.ConnectionHeaderResponse.parseFrom(buff); + + // Get the CryptoCipherMeta, update the HBaseSaslRpcClient for Crypto Cipher + if (connectionHeaderResponse.hasCryptoCipherMeta()) { + negotiateCryptoAes(connectionHeaderResponse.getCryptoCipherMeta()); + } + waitingConnectionHeaderResponse = false; + } catch (SocketTimeoutException ste) { + LOG.fatal("Can't get the connection header response for rpc timeout, please check if" + + " server has the correct configuration to support the additional function.", ste); + // timeout when waiting the connection header response, ignore the additional function + throw new IOException("Timeout while waiting connection header response", ste); + } + } + + private void negotiateCryptoAes(RPCProtos.CryptoCipherMeta cryptoCipherMeta) + throws IOException { + // initilize the Crypto AES with CryptoCipherMeta + saslRpcClient.initCryptoCipher(cryptoCipherMeta, this.rpcClient.conf); + // reset the inputStream/outputStream for Crypto AES encryption + this.in = new DataInputStream(new BufferedInputStream(saslRpcClient.getInputStream())); + this.out = new DataOutputStream(new BufferedOutputStream(saslRpcClient.getOutputStream())); + } + private void tracedWriteRequest(Call call) throws IOException { try (TraceScope ignored = Trace.startSpan("RpcClientImpl.tracedWriteRequest", call.span)) { writeRequest(call); 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 ce5addaf4fe..47d7234621e 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 @@ -22,6 +22,8 @@ import static org.apache.hadoop.hbase.ipc.CallEvent.Type.TIMEOUT; import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled; import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE; +import io.netty.handler.timeout.ReadTimeoutHandler; +import org.apache.hadoop.hbase.security.NettyHBaseRpcConnectionHeaderHandler; import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback; import io.netty.bootstrap.Bootstrap; @@ -55,7 +57,6 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; import org.apache.hadoop.hbase.security.NettyHBaseSaslRpcClientHandler; import org.apache.hadoop.hbase.security.SaslChallengeDecoder; -import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.security.UserGroupInformation; @@ -130,8 +131,7 @@ class NettyRpcConnection extends RpcConnection { } } - private void established(Channel ch) { - ch.write(connectionHeaderWithLength.retainedDuplicate()); + private void established(Channel ch) throws IOException { ChannelPipeline p = ch.pipeline(); String addBeforeHandler = p.context(BufferCallBeforeInitHandler.class).name(); p.addBefore(addBeforeHandler, null, @@ -188,11 +188,10 @@ class NettyRpcConnection extends RpcConnection { return; } Promise saslPromise = ch.eventLoop().newPromise(); - ChannelHandler saslHandler; + final NettyHBaseSaslRpcClientHandler saslHandler; try { saslHandler = new NettyHBaseSaslRpcClientHandler(saslPromise, ticket, authMethod, token, - serverPrincipal, rpcClient.fallbackAllowed, this.rpcClient.conf.get( - "hbase.rpc.protection", QualityOfProtection.AUTHENTICATION.name().toLowerCase())); + serverPrincipal, rpcClient.fallbackAllowed, this.rpcClient.conf); } catch (IOException e) { failInit(ch, e); return; @@ -206,7 +205,41 @@ class NettyRpcConnection extends RpcConnection { ChannelPipeline p = ch.pipeline(); p.remove(SaslChallengeDecoder.class); p.remove(NettyHBaseSaslRpcClientHandler.class); - established(ch); + + // check if negotiate with server for connection header is necessary + if (saslHandler.isNeedProcessConnectionHeader()) { + Promise connectionHeaderPromise = ch.eventLoop().newPromise(); + // create the handler to handle the connection header + ChannelHandler chHandler = new NettyHBaseRpcConnectionHeaderHandler( + connectionHeaderPromise, conf, connectionHeaderWithLength); + + // add ReadTimeoutHandler to deal with server doesn't response connection header + // because of the different configuration in client side and server side + p.addFirst(new ReadTimeoutHandler( + RpcClient.DEFAULT_SOCKET_TIMEOUT_READ, TimeUnit.MILLISECONDS)); + p.addLast(chHandler); + connectionHeaderPromise.addListener(new FutureListener() { + @Override + public void operationComplete(Future future) throws Exception { + if (future.isSuccess()) { + ChannelPipeline p = ch.pipeline(); + p.remove(ReadTimeoutHandler.class); + p.remove(NettyHBaseRpcConnectionHeaderHandler.class); + // don't send connection header, NettyHbaseRpcConnectionHeaderHandler + // sent it already + established(ch); + } else { + final Throwable error = future.cause(); + scheduleRelogin(error); + failInit(ch, toIOE(error)); + } + } + }); + } else { + // send the connection header to server + ch.write(connectionHeaderWithLength.retainedDuplicate()); + established(ch); + } } else { final Throwable error = future.cause(); scheduleRelogin(error); @@ -240,6 +273,8 @@ class NettyRpcConnection extends RpcConnection { if (useSasl) { saslNegotiate(ch); } else { + // send the connection header to server + ch.write(connectionHeaderWithLength.retainedDuplicate()); established(ch); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java index a60528e9952..c9002e793ed 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java @@ -72,6 +72,12 @@ abstract class RpcConnection { protected final HashedWheelTimer timeoutTimer; + protected final Configuration conf; + + protected static String CRYPTO_AES_ENABLED_KEY = "hbase.rpc.crypto.encryption.aes.enabled"; + + protected static boolean CRYPTO_AES_ENABLED_DEFAULT = false; + // the last time we were picked up from connection pool. protected long lastTouched; @@ -84,6 +90,7 @@ abstract class RpcConnection { this.timeoutTimer = timeoutTimer; this.codec = codec; this.compressor = compressor; + this.conf = conf; UserGroupInformation ticket = remoteId.getTicket().getUGI(); SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName()); @@ -224,6 +231,12 @@ abstract class RpcConnection { builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName()); } builder.setVersionInfo(ProtobufUtil.getVersionInfo()); + boolean isCryptoAESEnable = conf.getBoolean(CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT); + // if Crypto AES enable, setup Cipher transformation + if (isCryptoAESEnable) { + builder.setRpcCryptoCipherTransformation( + conf.get("hbase.rpc.crypto.encryption.aes.cipher.transform", "AES/CTR/NoPadding")); + } return builder.build(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java new file mode 100644 index 00000000000..12e4a7a81df --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java @@ -0,0 +1,38 @@ +/** + * 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 org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class UnsupportedCryptoException extends FatalConnectionException { + public UnsupportedCryptoException() { + super(); + } + + public UnsupportedCryptoException(String msg) { + super(msg); + } + + public UnsupportedCryptoException(String msg, Throwable t) { + super(msg, t); + } +} 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 new file mode 100644 index 00000000000..31abeba7b56 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java @@ -0,0 +1,47 @@ +/** + * 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 io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; + +/** + * Unwrap messages with Crypto AES. Should be placed after a + * {@link 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 new file mode 100644 index 00000000000..6c74ed8859c --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java @@ -0,0 +1,98 @@ +/** + * 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 io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.CoalescingBufferQueue; +import io.netty.util.ReferenceCountUtil; +import io.netty.util.concurrent.PromiseCombiner; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; + +import java.io.IOException; + +/** + * wrap messages with Crypto AES. + */ +@InterfaceAudience.Private +public class CryptoAESWrapHandler extends ChannelOutboundHandlerAdapter { + + private final CryptoAES cryptoAES; + + private CoalescingBufferQueue queue; + + public CryptoAESWrapHandler(CryptoAES cryptoAES) { + this.cryptoAES = cryptoAES; + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { + queue = new CoalescingBufferQueue(ctx.channel()); + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + if (msg instanceof ByteBuf) { + queue.add((ByteBuf) msg, promise); + } else { + ctx.write(msg, promise); + } + } + + @Override + public void flush(ChannelHandlerContext ctx) throws Exception { + if (queue.isEmpty()) { + return; + } + ByteBuf buf = null; + try { + ChannelPromise promise = ctx.newPromise(); + int readableBytes = queue.readableBytes(); + buf = queue.remove(readableBytes, promise); + byte[] bytes = new byte[readableBytes]; + buf.readBytes(bytes); + byte[] wrapperBytes = cryptoAES.wrap(bytes, 0, bytes.length); + ChannelPromise lenPromise = ctx.newPromise(); + ctx.write(ctx.alloc().buffer(4).writeInt(wrapperBytes.length), lenPromise); + ChannelPromise contentPromise = ctx.newPromise(); + ctx.write(Unpooled.wrappedBuffer(wrapperBytes), contentPromise); + PromiseCombiner combiner = new PromiseCombiner(); + combiner.addAll(lenPromise, contentPromise); + combiner.finish(promise); + ctx.flush(); + } finally { + if (buf != null) { + ReferenceCountUtil.safeRelease(buf); + } + } + } + + @Override + public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { + if (!queue.isEmpty()) { + queue.releaseAndFailAll(new IOException("Connection closed")); + } + ctx.close(promise); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java index b5009e0b060..c7e0be7ac3a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java @@ -23,9 +23,11 @@ import java.io.IOException; import java.security.Key; import java.security.KeyException; import java.security.SecureRandom; +import java.util.Properties; import javax.crypto.spec.SecretKeySpec; +import org.apache.commons.crypto.cipher.CryptoCipherFactory; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -37,6 +39,8 @@ import org.apache.hadoop.hbase.io.crypto.Cipher; import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; +import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; import org.apache.hadoop.hbase.util.Bytes; /** @@ -255,4 +259,27 @@ public final class EncryptionUtil { } return key; } + + /** + * Helper to create an instance of CryptoAES. + * + * @param conf The current configuration. + * @param cryptoCipherMeta The metadata for create CryptoAES. + * @return The instance of CryptoAES. + * @throws IOException if create CryptoAES failed + */ + public static CryptoAES createCryptoAES(RPCProtos.CryptoCipherMeta cryptoCipherMeta, + Configuration conf) throws IOException { + Properties properties = new Properties(); + // the property for cipher class + properties.setProperty(CryptoCipherFactory.CLASSES_KEY, + conf.get("hbase.rpc.crypto.encryption.aes.cipher.class", + "org.apache.commons.crypto.cipher.JceCipher")); + // create SaslAES for client + return new CryptoAES(cryptoCipherMeta.getTransformation(), properties, + cryptoCipherMeta.getInKey().toByteArray(), + cryptoCipherMeta.getOutKey().toByteArray(), + cryptoCipherMeta.getInIv().toByteArray(), + cryptoCipherMeta.getOutIv().toByteArray()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java index 3f43f7faf72..e644cb9dc5d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java @@ -22,16 +22,22 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; +import java.io.FilterInputStream; +import java.io.FilterOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; import javax.security.sasl.Sasl; import javax.security.sasl.SaslException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.security.SaslInputStream; @@ -47,6 +53,13 @@ import org.apache.hadoop.security.token.TokenIdentifier; public class HBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { private static final Log LOG = LogFactory.getLog(HBaseSaslRpcClient.class); + private boolean cryptoAesEnable; + private CryptoAES cryptoAES; + private InputStream saslInputStream; + private InputStream cryptoInputStream; + private OutputStream saslOutputStream; + private OutputStream cryptoOutputStream; + private boolean initStreamForCrypto; public HBaseSaslRpcClient(AuthMethod method, Token token, String serverPrincipal, boolean fallbackAllowed) throws IOException { @@ -54,8 +67,10 @@ public class HBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { } public HBaseSaslRpcClient(AuthMethod method, Token token, - String serverPrincipal, boolean fallbackAllowed, String rpcProtection) throws IOException { + String serverPrincipal, boolean fallbackAllowed, String rpcProtection, + boolean initStreamForCrypto) throws IOException { super(method, token, serverPrincipal, fallbackAllowed, rpcProtection); + this.initStreamForCrypto = initStreamForCrypto; } private static void readStatus(DataInputStream inStream) throws IOException { @@ -133,6 +148,18 @@ public class HBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { LOG.debug("SASL client context established. Negotiated QoP: " + saslClient.getNegotiatedProperty(Sasl.QOP)); } + // initial the inputStream, outputStream for both Sasl encryption + // and Crypto AES encryption if necessary + // if Crypto AES encryption enabled, the saslInputStream/saslOutputStream is + // only responsible for connection header negotiation, + // cryptoInputStream/cryptoOutputStream is responsible for rpc encryption with Crypto AES + saslInputStream = new SaslInputStream(inS, saslClient); + saslOutputStream = new SaslOutputStream(outS, saslClient); + if (initStreamForCrypto) { + cryptoInputStream = new WrappedInputStream(inS); + cryptoOutputStream = new WrappedOutputStream(outS); + } + return true; } catch (IOException e) { try { @@ -144,29 +171,112 @@ public class HBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { } } + public String getSaslQOP() { + return (String) saslClient.getNegotiatedProperty(Sasl.QOP); + } + + public void initCryptoCipher(RPCProtos.CryptoCipherMeta cryptoCipherMeta, + Configuration conf) throws IOException { + // create SaslAES for client + cryptoAES = EncryptionUtil.createCryptoAES(cryptoCipherMeta, conf); + cryptoAesEnable = true; + } + /** * Get a SASL wrapped InputStream. Can be called only after saslConnect() has been called. - * @param in the InputStream to wrap * @return a SASL wrapped InputStream * @throws IOException */ - public InputStream getInputStream(InputStream in) throws IOException { + public InputStream getInputStream() throws IOException { if (!saslClient.isComplete()) { throw new IOException("Sasl authentication exchange hasn't completed yet"); } - return new SaslInputStream(in, saslClient); + // If Crypto AES is enabled, return cryptoInputStream which unwrap the data with Crypto AES. + if (cryptoAesEnable && cryptoInputStream != null) { + return cryptoInputStream; + } + return saslInputStream; + } + + class WrappedInputStream extends FilterInputStream { + private ByteBuffer unwrappedRpcBuffer = ByteBuffer.allocate(0); + public WrappedInputStream(InputStream in) throws IOException { + super(in); + } + + @Override + public int read() throws IOException { + byte[] b = new byte[1]; + int n = read(b, 0, 1); + return (n != -1) ? b[0] : -1; + } + + @Override + public int read(byte b[]) throws IOException { + return read(b, 0, b.length); + } + + @Override + public synchronized int read(byte[] buf, int off, int len) throws IOException { + // fill the buffer with the next RPC message + if (unwrappedRpcBuffer.remaining() == 0) { + readNextRpcPacket(); + } + // satisfy as much of the request as possible + int readLen = Math.min(len, unwrappedRpcBuffer.remaining()); + unwrappedRpcBuffer.get(buf, off, readLen); + return readLen; + } + + // unwrap messages with Crypto AES + private void readNextRpcPacket() throws IOException { + LOG.debug("reading next wrapped RPC packet"); + DataInputStream dis = new DataInputStream(in); + int rpcLen = dis.readInt(); + byte[] rpcBuf = new byte[rpcLen]; + dis.readFully(rpcBuf); + + // unwrap with Crypto AES + rpcBuf = cryptoAES.unwrap(rpcBuf, 0, rpcBuf.length); + if (LOG.isDebugEnabled()) { + LOG.debug("unwrapping token of length:" + rpcBuf.length); + } + unwrappedRpcBuffer = ByteBuffer.wrap(rpcBuf); + } } /** * Get a SASL wrapped OutputStream. Can be called only after saslConnect() has been called. - * @param out the OutputStream to wrap * @return a SASL wrapped OutputStream * @throws IOException */ - public OutputStream getOutputStream(OutputStream out) throws IOException { + public OutputStream getOutputStream() throws IOException { if (!saslClient.isComplete()) { throw new IOException("Sasl authentication exchange hasn't completed yet"); } - return new SaslOutputStream(out, saslClient); + // If Crypto AES is enabled, return cryptoOutputStream which wrap the data with Crypto AES. + if (cryptoAesEnable && cryptoOutputStream != null) { + return cryptoOutputStream; + } + return saslOutputStream; + } + + class WrappedOutputStream extends FilterOutputStream { + public WrappedOutputStream(OutputStream out) throws IOException { + super(out); + } + @Override + public void write(byte[] buf, int off, int len) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("wrapping token of length:" + len); + } + + // wrap with Crypto AES + byte[] wrapped = cryptoAES.wrap(buf, off, len); + DataOutputStream dob = new DataOutputStream(out); + dob.writeInt(wrapped.length); + dob.write(wrapped, 0, wrapped.length); + dob.flush(); + } } } 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 new file mode 100644 index 00000000000..56088745905 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java @@ -0,0 +1,99 @@ +/** + * 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 io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.util.concurrent.Promise; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; + +/** + * Implement logic to deal with the rpc connection header. + */ +@InterfaceAudience.Private +public class NettyHBaseRpcConnectionHeaderHandler extends SimpleChannelInboundHandler { + + private final Promise saslPromise; + + private final Configuration conf; + + private final ByteBuf connectionHeaderWithLength; + + public NettyHBaseRpcConnectionHeaderHandler(Promise saslPromise, Configuration conf, + ByteBuf connectionHeaderWithLength) { + this.saslPromise = saslPromise; + this.conf = conf; + this.connectionHeaderWithLength = connectionHeaderWithLength; + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { + // read the ConnectionHeaderResponse from server + int len = msg.readInt(); + byte[] buff = new byte[len]; + msg.readBytes(buff); + + RPCProtos.ConnectionHeaderResponse connectionHeaderResponse = + RPCProtos.ConnectionHeaderResponse.parseFrom(buff); + + // Get the CryptoCipherMeta, update the HBaseSaslRpcClient for Crypto Cipher + if (connectionHeaderResponse.hasCryptoCipherMeta()) { + CryptoAES cryptoAES = EncryptionUtil.createCryptoAES( + connectionHeaderResponse.getCryptoCipherMeta(), conf); + // replace the Sasl handler with Crypto AES handler + setupCryptoAESHandler(ctx.pipeline(), cryptoAES); + } + + saslPromise.setSuccess(true); + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + try { + // send the connection header to server first + ctx.writeAndFlush(connectionHeaderWithLength.retainedDuplicate()); + } catch (Exception e) { + // the exception thrown by handlerAdded will not be passed to the exceptionCaught below + // because netty will remove a handler if handlerAdded throws an exception. + exceptionCaught(ctx, e); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + saslPromise.tryFailure(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)); + } +} 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 f624608e0d1..9ae31a4b000 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 @@ -47,6 +47,7 @@ public class NettyHBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { if (LOG.isDebugEnabled()) { LOG.debug("SASL client context established. Negotiated QoP: " + qop); } + if (qop == null || "auth".equalsIgnoreCase(qop)) { return; } @@ -55,4 +56,8 @@ public class NettyHBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4), new SaslUnwrapHandler(saslClient)); } + + public String getSaslQOP() { + return (String) saslClient.getNegotiatedProperty(Sasl.QOP); + } } 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 50609b4b3e1..4525aefcae2 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 @@ -27,6 +27,7 @@ import java.security.PrivilegedExceptionAction; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.ipc.FallbackDisallowedException; import org.apache.hadoop.security.UserGroupInformation; @@ -47,17 +48,25 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler< private final NettyHBaseSaslRpcClient saslRpcClient; + private final Configuration conf; + + // flag to mark if Crypto AES encryption is enable + private boolean needProcessConnectionHeader = false; + /** * @param saslPromise {@code true} if success, {@code false} if server tells us to fallback to * simple. */ public NettyHBaseSaslRpcClientHandler(Promise saslPromise, UserGroupInformation ugi, AuthMethod method, Token token, String serverPrincipal, - boolean fallbackAllowed, String rpcProtection) throws IOException { + boolean fallbackAllowed, Configuration conf) + throws IOException { this.saslPromise = saslPromise; this.ugi = ugi; + this.conf = conf; this.saslRpcClient = new NettyHBaseSaslRpcClient(method, token, serverPrincipal, - fallbackAllowed, rpcProtection); + fallbackAllowed, conf.get( + "hbase.rpc.protection", SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase())); } private void writeResponse(ChannelHandlerContext ctx, byte[] response) { @@ -72,10 +81,24 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler< if (!saslRpcClient.isComplete()) { return; } + saslRpcClient.setupSaslHandler(ctx.pipeline()); + setCryptoAESOption(); + saslPromise.setSuccess(true); } + private void setCryptoAESOption() { + boolean saslEncryptionEnabled = SaslUtil.QualityOfProtection.PRIVACY. + getSaslQop().equalsIgnoreCase(saslRpcClient.getSaslQOP()); + needProcessConnectionHeader = saslEncryptionEnabled && conf.getBoolean( + "hbase.rpc.crypto.encryption.aes.enabled", false); + } + + public boolean isNeedProcessConnectionHeader() { + return needProcessConnectionHeader; + } + @Override public void handlerAdded(ChannelHandlerContext ctx) { try { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java index 12b36616db8..7573a785577 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java @@ -88,7 +88,7 @@ public class TestHBaseSaslRpcClient { DEFAULT_USER_PASSWORD); for (SaslUtil.QualityOfProtection qop : SaslUtil.QualityOfProtection.values()) { String negotiatedQop = new HBaseSaslRpcClient(AuthMethod.DIGEST, token, - "principal/host@DOMAIN.COM", false, qop.name()) { + "principal/host@DOMAIN.COM", false, qop.name(), false) { public String getQop() { return saslProps.get(Sasl.QOP); } @@ -211,14 +211,14 @@ public class TestHBaseSaslRpcClient { }; try { - rpcClient.getInputStream(Mockito.mock(InputStream.class)); + rpcClient.getInputStream(); } catch(IOException ex) { //Sasl authentication exchange hasn't completed yet inState = true; } try { - rpcClient.getOutputStream(Mockito.mock(OutputStream.class)); + rpcClient.getOutputStream(); } catch(IOException ex) { //Sasl authentication exchange hasn't completed yet outState = true; diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index c5f5a815a4e..03f1682415a 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -275,6 +275,10 @@ org.apache.htrace htrace-core + + org.apache.commons + commons-crypto + diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CryptoAES.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CryptoAES.java new file mode 100644 index 00000000000..57ce2e1d8de --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CryptoAES.java @@ -0,0 +1,241 @@ +/** + * 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.aes; + +import org.apache.commons.crypto.cipher.CryptoCipher; +import org.apache.commons.crypto.utils.Utils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +import javax.crypto.Cipher; +import javax.crypto.Mac; +import javax.crypto.SecretKey; +import javax.crypto.ShortBufferException; +import javax.crypto.spec.IvParameterSpec; +import javax.crypto.spec.SecretKeySpec; +import javax.security.sasl.SaslException; +import java.io.IOException; +import java.security.InvalidAlgorithmParameterException; +import java.security.InvalidKeyException; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.Properties; + +/** + * AES encryption and decryption. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class CryptoAES { + + private final CryptoCipher encryptor; + private final CryptoCipher decryptor; + + private final Integrity integrity; + + public CryptoAES(String transformation, Properties properties, + byte[] inKey, byte[] outKey, byte[] inIv, byte[] outIv) throws IOException { + checkTransformation(transformation); + // encryptor + encryptor = Utils.getCipherInstance(transformation, properties); + try { + SecretKeySpec outKEYSpec = new SecretKeySpec(outKey, "AES"); + IvParameterSpec outIVSpec = new IvParameterSpec(outIv); + encryptor.init(Cipher.ENCRYPT_MODE, outKEYSpec, outIVSpec); + } catch (InvalidKeyException | InvalidAlgorithmParameterException e) { + throw new IOException("Failed to initialize encryptor", e); + } + + // decryptor + decryptor = Utils.getCipherInstance(transformation, properties); + try { + SecretKeySpec inKEYSpec = new SecretKeySpec(inKey, "AES"); + IvParameterSpec inIVSpec = new IvParameterSpec(inIv); + decryptor.init(Cipher.DECRYPT_MODE, inKEYSpec, inIVSpec); + } catch (InvalidKeyException | InvalidAlgorithmParameterException e) { + throw new IOException("Failed to initialize decryptor", e); + } + + integrity = new Integrity(outKey, inKey); + } + + /** + * Encrypts input data. The result composes of (msg, padding if needed, mac) and sequence num. + * @param data the input byte array + * @param offset the offset in input where the input starts + * @param len the input length + * @return the new encrypted byte array. + * @throws SaslException if error happens + */ + public byte[] wrap(byte[] data, int offset, int len) throws SaslException { + // mac + byte[] mac = integrity.getHMAC(data, offset, len); + integrity.incMySeqNum(); + + // encrypt + byte[] encrypted = new byte[len + 10]; + try { + int n = encryptor.update(data, offset, len, encrypted, 0); + encryptor.update(mac, 0, 10, encrypted, n); + } catch (ShortBufferException sbe) { + // this should not happen + throw new SaslException("Error happens during encrypt data", sbe); + } + + // append seqNum used for mac + byte[] wrapped = new byte[encrypted.length + 4]; + System.arraycopy(encrypted, 0, wrapped, 0, encrypted.length); + System.arraycopy(integrity.getSeqNum(), 0, wrapped, encrypted.length, 4); + + return wrapped; + } + + /** + * Decrypts input data. The input composes of (msg, padding if needed, mac) and sequence num. + * The result is msg. + * @param data the input byte array + * @param offset the offset in input where the input starts + * @param len the input length + * @return the new decrypted byte array. + * @throws SaslException if error happens + */ + public byte[] unwrap(byte[] data, int offset, int len) throws SaslException { + // get plaintext and seqNum + byte[] decrypted = new byte[len - 4]; + byte[] peerSeqNum = new byte[4]; + try { + decryptor.update(data, offset, len - 4, decrypted, 0); + } catch (ShortBufferException sbe) { + // this should not happen + throw new SaslException("Error happens during decrypt data", sbe); + } + System.arraycopy(data, offset + decrypted.length, peerSeqNum, 0, 4); + + // get msg and mac + byte[] msg = new byte[decrypted.length - 10]; + byte[] mac = new byte[10]; + System.arraycopy(decrypted, 0, msg, 0, msg.length); + System.arraycopy(decrypted, msg.length, mac, 0, 10); + + // check mac integrity and msg sequence + if (!integrity.compareHMAC(mac, peerSeqNum, msg, 0, msg.length)) { + throw new SaslException("Unmatched MAC"); + } + if (!integrity.comparePeerSeqNum(peerSeqNum)) { + throw new SaslException("Out of order sequencing of messages. Got: " + integrity.byteToInt + (peerSeqNum) + " Expected: " + integrity.peerSeqNum); + } + integrity.incPeerSeqNum(); + + return msg; + } + + private void checkTransformation(String transformation) throws IOException { + if ("AES/CTR/NoPadding".equalsIgnoreCase(transformation)) { + return; + } + throw new IOException("AES cipher transformation is not supported: " + transformation); + } + + /** + * Helper class for providing integrity protection. + */ + private static class Integrity { + + private int mySeqNum = 0; + private int peerSeqNum = 0; + private byte[] seqNum = new byte[4]; + + private byte[] myKey; + private byte[] peerKey; + + Integrity(byte[] outKey, byte[] inKey) throws IOException { + myKey = outKey; + peerKey = inKey; + } + + byte[] getHMAC(byte[] msg, int start, int len) throws SaslException { + intToByte(mySeqNum); + return calculateHMAC(myKey, seqNum, msg, start, len); + } + + boolean compareHMAC(byte[] expectedHMAC, byte[] peerSeqNum, byte[] msg, int start, + int len) throws SaslException { + byte[] mac = calculateHMAC(peerKey, peerSeqNum, msg, start, len); + return Arrays.equals(mac, expectedHMAC); + } + + boolean comparePeerSeqNum(byte[] peerSeqNum) { + return this.peerSeqNum == byteToInt(peerSeqNum); + } + + byte[] getSeqNum() { + return seqNum; + } + + void incMySeqNum() { + mySeqNum ++; + } + + void incPeerSeqNum() { + peerSeqNum ++; + } + + private byte[] calculateHMAC(byte[] key, byte[] seqNum, byte[] msg, int start, + int len) throws SaslException { + byte[] seqAndMsg = new byte[4+len]; + System.arraycopy(seqNum, 0, seqAndMsg, 0, 4); + System.arraycopy(msg, start, seqAndMsg, 4, len); + + try { + SecretKey keyKi = new SecretKeySpec(key, "HmacMD5"); + Mac m = Mac.getInstance("HmacMD5"); + m.init(keyKi); + m.update(seqAndMsg); + byte[] hMAC_MD5 = m.doFinal(); + + /* First 10 bytes of HMAC_MD5 digest */ + byte macBuffer[] = new byte[10]; + System.arraycopy(hMAC_MD5, 0, macBuffer, 0, 10); + + return macBuffer; + } catch (InvalidKeyException e) { + throw new SaslException("Invalid bytes used for key of HMAC-MD5 hash.", e); + } catch (NoSuchAlgorithmException e) { + throw new SaslException("Error creating instance of MD5 MAC algorithm", e); + } + } + + private void intToByte(int num) { + for(int i = 3; i >= 0; i --) { + seqNum[i] = (byte)(num & 0xff); + num >>>= 8; + } + } + + private int byteToInt(byte[] seqNum) { + int answer = 0; + for (int i = 0; i < 4; i ++) { + answer <<= 8; + answer |= ((int)seqNum[i] & 0xff); + } + return answer; + } + } +} diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java index 3c2bec0cadf..d12b30e00c9 100644 --- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java @@ -839,6 +839,32 @@ public final class RPCProtos { * optional .hbase.pb.VersionInfo version_info = 5; */ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder getVersionInfoOrBuilder(); + + /** + *
+     * the transformation for rpc AES encryption with Apache Commons Crypto
+     * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + boolean hasRpcCryptoCipherTransformation(); + /** + *
+     * the transformation for rpc AES encryption with Apache Commons Crypto
+     * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + java.lang.String getRpcCryptoCipherTransformation(); + /** + *
+     * the transformation for rpc AES encryption with Apache Commons Crypto
+     * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getRpcCryptoCipherTransformationBytes(); } /** *
@@ -859,6 +885,7 @@ public final class RPCProtos {
       serviceName_ = "";
       cellBlockCodecClass_ = "";
       cellBlockCompressorClass_ = "";
+      rpcCryptoCipherTransformation_ = "";
     }
 
     @java.lang.Override
@@ -933,6 +960,12 @@ public final class RPCProtos {
               bitField0_ |= 0x00000010;
               break;
             }
+            case 50: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000020;
+              rpcCryptoCipherTransformation_ = bs;
+              break;
+            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -1156,6 +1189,60 @@ public final class RPCProtos {
       return versionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo.getDefaultInstance() : versionInfo_;
     }
 
+    public static final int RPC_CRYPTO_CIPHER_TRANSFORMATION_FIELD_NUMBER = 6;
+    private volatile java.lang.Object rpcCryptoCipherTransformation_;
+    /**
+     * 
+     * the transformation for rpc AES encryption with Apache Commons Crypto
+     * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public boolean hasRpcCryptoCipherTransformation() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + *
+     * the transformation for rpc AES encryption with Apache Commons Crypto
+     * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public java.lang.String getRpcCryptoCipherTransformation() { + java.lang.Object ref = rpcCryptoCipherTransformation_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + rpcCryptoCipherTransformation_ = s; + } + return s; + } + } + /** + *
+     * the transformation for rpc AES encryption with Apache Commons Crypto
+     * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getRpcCryptoCipherTransformationBytes() { + java.lang.Object ref = rpcCryptoCipherTransformation_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + rpcCryptoCipherTransformation_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; @@ -1195,6 +1282,9 @@ public final class RPCProtos { if (((bitField0_ & 0x00000010) == 0x00000010)) { output.writeMessage(5, getVersionInfo()); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 6, rpcCryptoCipherTransformation_); + } unknownFields.writeTo(output); } @@ -1220,6 +1310,9 @@ public final class RPCProtos { size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream .computeMessageSize(5, getVersionInfo()); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(6, rpcCryptoCipherTransformation_); + } size += unknownFields.getSerializedSize(); memoizedSize = size; return size; @@ -1262,6 +1355,11 @@ public final class RPCProtos { result = result && getVersionInfo() .equals(other.getVersionInfo()); } + result = result && (hasRpcCryptoCipherTransformation() == other.hasRpcCryptoCipherTransformation()); + if (hasRpcCryptoCipherTransformation()) { + result = result && getRpcCryptoCipherTransformation() + .equals(other.getRpcCryptoCipherTransformation()); + } result = result && unknownFields.equals(other.unknownFields); return result; } @@ -1293,6 +1391,10 @@ public final class RPCProtos { hash = (37 * hash) + VERSION_INFO_FIELD_NUMBER; hash = (53 * hash) + getVersionInfo().hashCode(); } + if (hasRpcCryptoCipherTransformation()) { + hash = (37 * hash) + RPC_CRYPTO_CIPHER_TRANSFORMATION_FIELD_NUMBER; + hash = (53 * hash) + getRpcCryptoCipherTransformation().hashCode(); + } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; return hash; @@ -1435,6 +1537,8 @@ public final class RPCProtos { versionInfoBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000010); + rpcCryptoCipherTransformation_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); return this; } @@ -1487,6 +1591,10 @@ public final class RPCProtos { } else { result.versionInfo_ = versionInfoBuilder_.build(); } + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.rpcCryptoCipherTransformation_ = rpcCryptoCipherTransformation_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1550,6 +1658,11 @@ public final class RPCProtos { if (other.hasVersionInfo()) { mergeVersionInfo(other.getVersionInfo()); } + if (other.hasRpcCryptoCipherTransformation()) { + bitField0_ |= 0x00000020; + rpcCryptoCipherTransformation_ = other.rpcCryptoCipherTransformation_; + onChanged(); + } this.mergeUnknownFields(other.unknownFields); onChanged(); return this; @@ -2111,6 +2224,106 @@ public final class RPCProtos { } return versionInfoBuilder_; } + + private java.lang.Object rpcCryptoCipherTransformation_ = ""; + /** + *
+       * the transformation for rpc AES encryption with Apache Commons Crypto
+       * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public boolean hasRpcCryptoCipherTransformation() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + *
+       * the transformation for rpc AES encryption with Apache Commons Crypto
+       * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public java.lang.String getRpcCryptoCipherTransformation() { + java.lang.Object ref = rpcCryptoCipherTransformation_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + rpcCryptoCipherTransformation_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + *
+       * the transformation for rpc AES encryption with Apache Commons Crypto
+       * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getRpcCryptoCipherTransformationBytes() { + java.lang.Object ref = rpcCryptoCipherTransformation_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + rpcCryptoCipherTransformation_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + *
+       * the transformation for rpc AES encryption with Apache Commons Crypto
+       * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public Builder setRpcCryptoCipherTransformation( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + rpcCryptoCipherTransformation_ = value; + onChanged(); + return this; + } + /** + *
+       * the transformation for rpc AES encryption with Apache Commons Crypto
+       * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public Builder clearRpcCryptoCipherTransformation() { + bitField0_ = (bitField0_ & ~0x00000020); + rpcCryptoCipherTransformation_ = getDefaultInstance().getRpcCryptoCipherTransformation(); + onChanged(); + return this; + } + /** + *
+       * the transformation for rpc AES encryption with Apache Commons Crypto
+       * 
+ * + * optional string rpc_crypto_cipher_transformation = 6; + */ + public Builder setRpcCryptoCipherTransformationBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + rpcCryptoCipherTransformation_ = value; + onChanged(); + return this; + } public final Builder setUnknownFields( final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); @@ -2160,6 +2373,663 @@ public final class RPCProtos { } + public interface ConnectionHeaderResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.ConnectionHeaderResponse) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + *
+     * To use Apache Commons Crypto, negotiate the metadata
+     * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + boolean hasCryptoCipherMeta(); + /** + *
+     * To use Apache Commons Crypto, negotiate the metadata
+     * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta getCryptoCipherMeta(); + /** + *
+     * To use Apache Commons Crypto, negotiate the metadata
+     * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMetaOrBuilder getCryptoCipherMetaOrBuilder(); + } + /** + *
+   * This is sent by rpc server to negotiate the data if necessary
+   * 
+ * + * Protobuf type {@code hbase.pb.ConnectionHeaderResponse} + */ + public static final class ConnectionHeaderResponse extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.ConnectionHeaderResponse) + ConnectionHeaderResponseOrBuilder { + // Use ConnectionHeaderResponse.newBuilder() to construct. + private ConnectionHeaderResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private ConnectionHeaderResponse() { + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ConnectionHeaderResponse( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = cryptoCipherMeta_.toBuilder(); + } + cryptoCipherMeta_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(cryptoCipherMeta_); + cryptoCipherMeta_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeaderResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeaderResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse.Builder.class); + } + + private int bitField0_; + public static final int CRYPTO_CIPHER_META_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta cryptoCipherMeta_; + /** + *
+     * To use Apache Commons Crypto, negotiate the metadata
+     * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public boolean hasCryptoCipherMeta() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + *
+     * To use Apache Commons Crypto, negotiate the metadata
+     * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta getCryptoCipherMeta() { + return cryptoCipherMeta_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.getDefaultInstance() : cryptoCipherMeta_; + } + /** + *
+     * To use Apache Commons Crypto, negotiate the metadata
+     * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMetaOrBuilder getCryptoCipherMetaOrBuilder() { + return cryptoCipherMeta_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.getDefaultInstance() : cryptoCipherMeta_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (hasCryptoCipherMeta()) { + if (!getCryptoCipherMeta().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, getCryptoCipherMeta()); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getCryptoCipherMeta()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse) obj; + + boolean result = true; + result = result && (hasCryptoCipherMeta() == other.hasCryptoCipherMeta()); + if (hasCryptoCipherMeta()) { + result = result && getCryptoCipherMeta() + .equals(other.getCryptoCipherMeta()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasCryptoCipherMeta()) { + hash = (37 * hash) + CRYPTO_CIPHER_META_FIELD_NUMBER; + hash = (53 * hash) + getCryptoCipherMeta().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+     * This is sent by rpc server to negotiate the data if necessary
+     * 
+ * + * Protobuf type {@code hbase.pb.ConnectionHeaderResponse} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.ConnectionHeaderResponse) + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponseOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeaderResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeaderResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getCryptoCipherMetaFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (cryptoCipherMetaBuilder_ == null) { + cryptoCipherMeta_ = null; + } else { + cryptoCipherMetaBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeaderResponse_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (cryptoCipherMetaBuilder_ == null) { + result.cryptoCipherMeta_ = cryptoCipherMeta_; + } else { + result.cryptoCipherMeta_ = cryptoCipherMetaBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse.getDefaultInstance()) return this; + if (other.hasCryptoCipherMeta()) { + mergeCryptoCipherMeta(other.getCryptoCipherMeta()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (hasCryptoCipherMeta()) { + if (!getCryptoCipherMeta().isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta cryptoCipherMeta_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMetaOrBuilder> cryptoCipherMetaBuilder_; + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public boolean hasCryptoCipherMeta() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta getCryptoCipherMeta() { + if (cryptoCipherMetaBuilder_ == null) { + return cryptoCipherMeta_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.getDefaultInstance() : cryptoCipherMeta_; + } else { + return cryptoCipherMetaBuilder_.getMessage(); + } + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public Builder setCryptoCipherMeta(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta value) { + if (cryptoCipherMetaBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + cryptoCipherMeta_ = value; + onChanged(); + } else { + cryptoCipherMetaBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public Builder setCryptoCipherMeta( + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder builderForValue) { + if (cryptoCipherMetaBuilder_ == null) { + cryptoCipherMeta_ = builderForValue.build(); + onChanged(); + } else { + cryptoCipherMetaBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public Builder mergeCryptoCipherMeta(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta value) { + if (cryptoCipherMetaBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + cryptoCipherMeta_ != null && + cryptoCipherMeta_ != org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.getDefaultInstance()) { + cryptoCipherMeta_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.newBuilder(cryptoCipherMeta_).mergeFrom(value).buildPartial(); + } else { + cryptoCipherMeta_ = value; + } + onChanged(); + } else { + cryptoCipherMetaBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public Builder clearCryptoCipherMeta() { + if (cryptoCipherMetaBuilder_ == null) { + cryptoCipherMeta_ = null; + onChanged(); + } else { + cryptoCipherMetaBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder getCryptoCipherMetaBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getCryptoCipherMetaFieldBuilder().getBuilder(); + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMetaOrBuilder getCryptoCipherMetaOrBuilder() { + if (cryptoCipherMetaBuilder_ != null) { + return cryptoCipherMetaBuilder_.getMessageOrBuilder(); + } else { + return cryptoCipherMeta_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.getDefaultInstance() : cryptoCipherMeta_; + } + } + /** + *
+       * To use Apache Commons Crypto, negotiate the metadata
+       * 
+ * + * optional .hbase.pb.CryptoCipherMeta crypto_cipher_meta = 1; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMetaOrBuilder> + getCryptoCipherMetaFieldBuilder() { + if (cryptoCipherMetaBuilder_ == null) { + cryptoCipherMetaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMetaOrBuilder>( + getCryptoCipherMeta(), + getParentForChildren(), + isClean()); + cryptoCipherMeta_ = null; + } + return cryptoCipherMetaBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.ConnectionHeaderResponse) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.ConnectionHeaderResponse) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public ConnectionHeaderResponse parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new ConnectionHeaderResponse(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + public interface CellBlockMetaOrBuilder extends // @@protoc_insertion_point(interface_extends:hbase.pb.CellBlockMeta) org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { @@ -3913,6 +4783,927 @@ public final class RPCProtos { } + public interface CryptoCipherMetaOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.CryptoCipherMeta) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required string transformation = 1; + */ + boolean hasTransformation(); + /** + * required string transformation = 1; + */ + java.lang.String getTransformation(); + /** + * required string transformation = 1; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getTransformationBytes(); + + /** + * optional bytes inKey = 2; + */ + boolean hasInKey(); + /** + * optional bytes inKey = 2; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getInKey(); + + /** + * optional bytes inIv = 3; + */ + boolean hasInIv(); + /** + * optional bytes inIv = 3; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getInIv(); + + /** + * optional bytes outKey = 4; + */ + boolean hasOutKey(); + /** + * optional bytes outKey = 4; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getOutKey(); + + /** + * optional bytes outIv = 5; + */ + boolean hasOutIv(); + /** + * optional bytes outIv = 5; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getOutIv(); + } + /** + *
+   **
+   * Cipher meta for Crypto
+   * 
+ * + * Protobuf type {@code hbase.pb.CryptoCipherMeta} + */ + public static final class CryptoCipherMeta extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.CryptoCipherMeta) + CryptoCipherMetaOrBuilder { + // Use CryptoCipherMeta.newBuilder() to construct. + private CryptoCipherMeta(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CryptoCipherMeta() { + transformation_ = ""; + inKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + inIv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + outKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + outIv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CryptoCipherMeta( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + transformation_ = bs; + break; + } + case 18: { + bitField0_ |= 0x00000002; + inKey_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + inIv_ = input.readBytes(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + outKey_ = input.readBytes(); + break; + } + case 42: { + bitField0_ |= 0x00000010; + outIv_ = input.readBytes(); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CryptoCipherMeta_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CryptoCipherMeta_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder.class); + } + + private int bitField0_; + public static final int TRANSFORMATION_FIELD_NUMBER = 1; + private volatile java.lang.Object transformation_; + /** + * required string transformation = 1; + */ + public boolean hasTransformation() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string transformation = 1; + */ + public java.lang.String getTransformation() { + java.lang.Object ref = transformation_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + transformation_ = s; + } + return s; + } + } + /** + * required string transformation = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getTransformationBytes() { + java.lang.Object ref = transformation_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + transformation_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + public static final int INKEY_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString inKey_; + /** + * optional bytes inKey = 2; + */ + public boolean hasInKey() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes inKey = 2; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getInKey() { + return inKey_; + } + + public static final int INIV_FIELD_NUMBER = 3; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString inIv_; + /** + * optional bytes inIv = 3; + */ + public boolean hasInIv() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional bytes inIv = 3; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getInIv() { + return inIv_; + } + + public static final int OUTKEY_FIELD_NUMBER = 4; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString outKey_; + /** + * optional bytes outKey = 4; + */ + public boolean hasOutKey() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional bytes outKey = 4; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getOutKey() { + return outKey_; + } + + public static final int OUTIV_FIELD_NUMBER = 5; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString outIv_; + /** + * optional bytes outIv = 5; + */ + public boolean hasOutIv() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional bytes outIv = 5; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getOutIv() { + return outIv_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasTransformation()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, transformation_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, inKey_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, inIv_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, outKey_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, outIv_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, transformation_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeBytesSize(2, inKey_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeBytesSize(3, inIv_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeBytesSize(4, outKey_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeBytesSize(5, outIv_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta other = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta) obj; + + boolean result = true; + result = result && (hasTransformation() == other.hasTransformation()); + if (hasTransformation()) { + result = result && getTransformation() + .equals(other.getTransformation()); + } + result = result && (hasInKey() == other.hasInKey()); + if (hasInKey()) { + result = result && getInKey() + .equals(other.getInKey()); + } + result = result && (hasInIv() == other.hasInIv()); + if (hasInIv()) { + result = result && getInIv() + .equals(other.getInIv()); + } + result = result && (hasOutKey() == other.hasOutKey()); + if (hasOutKey()) { + result = result && getOutKey() + .equals(other.getOutKey()); + } + result = result && (hasOutIv() == other.hasOutIv()); + if (hasOutIv()) { + result = result && getOutIv() + .equals(other.getOutIv()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasTransformation()) { + hash = (37 * hash) + TRANSFORMATION_FIELD_NUMBER; + hash = (53 * hash) + getTransformation().hashCode(); + } + if (hasInKey()) { + hash = (37 * hash) + INKEY_FIELD_NUMBER; + hash = (53 * hash) + getInKey().hashCode(); + } + if (hasInIv()) { + hash = (37 * hash) + INIV_FIELD_NUMBER; + hash = (53 * hash) + getInIv().hashCode(); + } + if (hasOutKey()) { + hash = (37 * hash) + OUTKEY_FIELD_NUMBER; + hash = (53 * hash) + getOutKey().hashCode(); + } + if (hasOutIv()) { + hash = (37 * hash) + OUTIV_FIELD_NUMBER; + hash = (53 * hash) + getOutIv().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+     **
+     * Cipher meta for Crypto
+     * 
+ * + * Protobuf type {@code hbase.pb.CryptoCipherMeta} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.CryptoCipherMeta) + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMetaOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CryptoCipherMeta_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CryptoCipherMeta_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + transformation_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + inKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + inIv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + outKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + outIv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CryptoCipherMeta_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta result = new org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.transformation_ = transformation_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.inKey_ = inKey_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.inIv_ = inIv_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.outKey_ = outKey_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.outIv_ = outIv_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta.getDefaultInstance()) return this; + if (other.hasTransformation()) { + bitField0_ |= 0x00000001; + transformation_ = other.transformation_; + onChanged(); + } + if (other.hasInKey()) { + setInKey(other.getInKey()); + } + if (other.hasInIv()) { + setInIv(other.getInIv()); + } + if (other.hasOutKey()) { + setOutKey(other.getOutKey()); + } + if (other.hasOutIv()) { + setOutIv(other.getOutIv()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasTransformation()) { + return false; + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.lang.Object transformation_ = ""; + /** + * required string transformation = 1; + */ + public boolean hasTransformation() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string transformation = 1; + */ + public java.lang.String getTransformation() { + java.lang.Object ref = transformation_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + transformation_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string transformation = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getTransformationBytes() { + java.lang.Object ref = transformation_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + transformation_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * required string transformation = 1; + */ + public Builder setTransformation( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + transformation_ = value; + onChanged(); + return this; + } + /** + * required string transformation = 1; + */ + public Builder clearTransformation() { + bitField0_ = (bitField0_ & ~0x00000001); + transformation_ = getDefaultInstance().getTransformation(); + onChanged(); + return this; + } + /** + * required string transformation = 1; + */ + public Builder setTransformationBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + transformation_ = value; + onChanged(); + return this; + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString inKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes inKey = 2; + */ + public boolean hasInKey() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes inKey = 2; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getInKey() { + return inKey_; + } + /** + * optional bytes inKey = 2; + */ + public Builder setInKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + inKey_ = value; + onChanged(); + return this; + } + /** + * optional bytes inKey = 2; + */ + public Builder clearInKey() { + bitField0_ = (bitField0_ & ~0x00000002); + inKey_ = getDefaultInstance().getInKey(); + onChanged(); + return this; + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString inIv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes inIv = 3; + */ + public boolean hasInIv() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional bytes inIv = 3; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getInIv() { + return inIv_; + } + /** + * optional bytes inIv = 3; + */ + public Builder setInIv(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + inIv_ = value; + onChanged(); + return this; + } + /** + * optional bytes inIv = 3; + */ + public Builder clearInIv() { + bitField0_ = (bitField0_ & ~0x00000004); + inIv_ = getDefaultInstance().getInIv(); + onChanged(); + return this; + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString outKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes outKey = 4; + */ + public boolean hasOutKey() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional bytes outKey = 4; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getOutKey() { + return outKey_; + } + /** + * optional bytes outKey = 4; + */ + public Builder setOutKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + outKey_ = value; + onChanged(); + return this; + } + /** + * optional bytes outKey = 4; + */ + public Builder clearOutKey() { + bitField0_ = (bitField0_ & ~0x00000008); + outKey_ = getDefaultInstance().getOutKey(); + onChanged(); + return this; + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString outIv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes outIv = 5; + */ + public boolean hasOutIv() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional bytes outIv = 5; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getOutIv() { + return outIv_; + } + /** + * optional bytes outIv = 5; + */ + public Builder setOutIv(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + outIv_ = value; + onChanged(); + return this; + } + /** + * optional bytes outIv = 5; + */ + public Builder clearOutIv() { + bitField0_ = (bitField0_ & ~0x00000010); + outIv_ = getDefaultInstance().getOutIv(); + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.CryptoCipherMeta) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.CryptoCipherMeta) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public CryptoCipherMeta parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new CryptoCipherMeta(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CryptoCipherMeta getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + public interface RequestHeaderOrBuilder extends // @@protoc_insertion_point(interface_extends:hbase.pb.RequestHeader) org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { @@ -6370,6 +8161,11 @@ public final class RPCProtos { private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_ConnectionHeader_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_ConnectionHeaderResponse_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_ConnectionHeaderResponse_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_CellBlockMeta_descriptor; private static final @@ -6380,6 +8176,11 @@ public final class RPCProtos { private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_ExceptionResponse_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_CryptoCipherMeta_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_CryptoCipherMeta_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_RequestHeader_descriptor; private static final @@ -6401,26 +8202,32 @@ public final class RPCProtos { java.lang.String[] descriptorData = { "\n\tRPC.proto\022\010hbase.pb\032\rTracing.proto\032\013HB" + "ase.proto\"<\n\017UserInformation\022\026\n\016effectiv" + - "e_user\030\001 \002(\t\022\021\n\treal_user\030\002 \001(\t\"\310\001\n\020Conn" + + "e_user\030\001 \002(\t\022\021\n\treal_user\030\002 \001(\t\"\362\001\n\020Conn" + "ectionHeader\022,\n\tuser_info\030\001 \001(\0132\031.hbase." + "pb.UserInformation\022\024\n\014service_name\030\002 \001(\t" + "\022\036\n\026cell_block_codec_class\030\003 \001(\t\022#\n\033cell" + "_block_compressor_class\030\004 \001(\t\022+\n\014version" + - "_info\030\005 \001(\0132\025.hbase.pb.VersionInfo\"\037\n\rCe" + - "llBlockMeta\022\016\n\006length\030\001 \001(\r\"|\n\021Exception" + - "Response\022\034\n\024exception_class_name\030\001 \001(\t\022\023", - "\n\013stack_trace\030\002 \001(\t\022\020\n\010hostname\030\003 \001(\t\022\014\n" + - "\004port\030\004 \001(\005\022\024\n\014do_not_retry\030\005 \001(\010\"\311\001\n\rRe" + - "questHeader\022\017\n\007call_id\030\001 \001(\r\022&\n\ntrace_in" + - "fo\030\002 \001(\0132\022.hbase.pb.RPCTInfo\022\023\n\013method_n" + - "ame\030\003 \001(\t\022\025\n\rrequest_param\030\004 \001(\010\0220\n\017cell" + - "_block_meta\030\005 \001(\0132\027.hbase.pb.CellBlockMe" + - "ta\022\020\n\010priority\030\006 \001(\r\022\017\n\007timeout\030\007 \001(\r\"\203\001" + - "\n\016ResponseHeader\022\017\n\007call_id\030\001 \001(\r\022.\n\texc" + - "eption\030\002 \001(\0132\033.hbase.pb.ExceptionRespons" + - "e\0220\n\017cell_block_meta\030\003 \001(\0132\027.hbase.pb.Ce", - "llBlockMetaBC\n1org.apache.hadoop.hbase.s" + - "haded.protobuf.generatedB\tRPCProtosH\001\240\001\001" + "_info\030\005 \001(\0132\025.hbase.pb.VersionInfo\022(\n rp" + + "c_crypto_cipher_transformation\030\006 \001(\t\"R\n\030" + + "ConnectionHeaderResponse\0226\n\022crypto_ciphe", + "r_meta\030\001 \001(\0132\032.hbase.pb.CryptoCipherMeta" + + "\"\037\n\rCellBlockMeta\022\016\n\006length\030\001 \001(\r\"|\n\021Exc" + + "eptionResponse\022\034\n\024exception_class_name\030\001" + + " \001(\t\022\023\n\013stack_trace\030\002 \001(\t\022\020\n\010hostname\030\003 " + + "\001(\t\022\014\n\004port\030\004 \001(\005\022\024\n\014do_not_retry\030\005 \001(\010\"" + + "f\n\020CryptoCipherMeta\022\026\n\016transformation\030\001 " + + "\002(\t\022\r\n\005inKey\030\002 \001(\014\022\014\n\004inIv\030\003 \001(\014\022\016\n\006outK" + + "ey\030\004 \001(\014\022\r\n\005outIv\030\005 \001(\014\"\311\001\n\rRequestHeade" + + "r\022\017\n\007call_id\030\001 \001(\r\022&\n\ntrace_info\030\002 \001(\0132\022" + + ".hbase.pb.RPCTInfo\022\023\n\013method_name\030\003 \001(\t\022", + "\025\n\rrequest_param\030\004 \001(\010\0220\n\017cell_block_met" + + "a\030\005 \001(\0132\027.hbase.pb.CellBlockMeta\022\020\n\010prio" + + "rity\030\006 \001(\r\022\017\n\007timeout\030\007 \001(\r\"\203\001\n\016Response" + + "Header\022\017\n\007call_id\030\001 \001(\r\022.\n\texception\030\002 \001" + + "(\0132\033.hbase.pb.ExceptionResponse\0220\n\017cell_" + + "block_meta\030\003 \001(\0132\027.hbase.pb.CellBlockMet" + + "aBC\n1org.apache.hadoop.hbase.shaded.prot" + + "obuf.generatedB\tRPCProtosH\001\240\001\001" }; org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -6447,27 +8254,39 @@ public final class RPCProtos { internal_static_hbase_pb_ConnectionHeader_fieldAccessorTable = new org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_hbase_pb_ConnectionHeader_descriptor, - new java.lang.String[] { "UserInfo", "ServiceName", "CellBlockCodecClass", "CellBlockCompressorClass", "VersionInfo", }); - internal_static_hbase_pb_CellBlockMeta_descriptor = + new java.lang.String[] { "UserInfo", "ServiceName", "CellBlockCodecClass", "CellBlockCompressorClass", "VersionInfo", "RpcCryptoCipherTransformation", }); + internal_static_hbase_pb_ConnectionHeaderResponse_descriptor = getDescriptor().getMessageTypes().get(2); + internal_static_hbase_pb_ConnectionHeaderResponse_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_ConnectionHeaderResponse_descriptor, + new java.lang.String[] { "CryptoCipherMeta", }); + internal_static_hbase_pb_CellBlockMeta_descriptor = + getDescriptor().getMessageTypes().get(3); internal_static_hbase_pb_CellBlockMeta_fieldAccessorTable = new org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_hbase_pb_CellBlockMeta_descriptor, new java.lang.String[] { "Length", }); internal_static_hbase_pb_ExceptionResponse_descriptor = - getDescriptor().getMessageTypes().get(3); + getDescriptor().getMessageTypes().get(4); internal_static_hbase_pb_ExceptionResponse_fieldAccessorTable = new org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_hbase_pb_ExceptionResponse_descriptor, new java.lang.String[] { "ExceptionClassName", "StackTrace", "Hostname", "Port", "DoNotRetry", }); + internal_static_hbase_pb_CryptoCipherMeta_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_hbase_pb_CryptoCipherMeta_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_CryptoCipherMeta_descriptor, + new java.lang.String[] { "Transformation", "InKey", "InIv", "OutKey", "OutIv", }); internal_static_hbase_pb_RequestHeader_descriptor = - getDescriptor().getMessageTypes().get(4); + getDescriptor().getMessageTypes().get(6); internal_static_hbase_pb_RequestHeader_fieldAccessorTable = new org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_hbase_pb_RequestHeader_descriptor, new java.lang.String[] { "CallId", "TraceInfo", "MethodName", "RequestParam", "CellBlockMeta", "Priority", "Timeout", }); internal_static_hbase_pb_ResponseHeader_descriptor = - getDescriptor().getMessageTypes().get(5); + getDescriptor().getMessageTypes().get(7); internal_static_hbase_pb_ResponseHeader_fieldAccessorTable = new org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_hbase_pb_ResponseHeader_descriptor, diff --git a/hbase-protocol-shaded/src/main/protobuf/RPC.proto b/hbase-protocol-shaded/src/main/protobuf/RPC.proto index 0cb234d4db8..9cdf98c7bd2 100644 --- a/hbase-protocol-shaded/src/main/protobuf/RPC.proto +++ b/hbase-protocol-shaded/src/main/protobuf/RPC.proto @@ -89,6 +89,14 @@ message ConnectionHeader { // Class must implement hadoop's CompressionCodec Interface. Can't compress if no codec. optional string cell_block_compressor_class = 4; optional VersionInfo version_info = 5; + // the transformation for rpc AES encryption with Apache Commons Crypto + optional string rpc_crypto_cipher_transformation = 6; +} + +// This is sent by rpc server to negotiate the data if necessary +message ConnectionHeaderResponse { + // To use Apache Commons Crypto, negotiate the metadata + optional CryptoCipherMeta crypto_cipher_meta = 1; } // Optional Cell block Message. Included in client RequestHeader @@ -112,6 +120,17 @@ message ExceptionResponse { optional bool do_not_retry = 5; } +/** + * Cipher meta for Crypto + */ +message CryptoCipherMeta { + required string transformation = 1; + optional bytes inKey = 2; + optional bytes inIv = 3; + optional bytes outKey = 4; + optional bytes outIv = 5; +} + // Header sent making a request. message RequestHeader { // Monotonically increasing call_id to keep track of RPC requests and their response diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 836c4bcdcf5..00c7254944c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -45,6 +45,7 @@ import java.nio.channels.Selector; import java.nio.channels.ServerSocketChannel; import java.nio.channels.SocketChannel; import java.nio.channels.WritableByteChannel; +import java.security.GeneralSecurityException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; @@ -53,6 +54,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.Timer; import java.util.TimerTask; @@ -70,6 +72,9 @@ import javax.security.sasl.Sasl; import javax.security.sasl.SaslException; import javax.security.sasl.SaslServer; +import org.apache.commons.crypto.cipher.CryptoCipherFactory; +import org.apache.commons.crypto.random.CryptoRandom; +import org.apache.commons.crypto.random.CryptoRandomFactory; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -91,10 +96,12 @@ import org.apache.hadoop.hbase.io.ByteBufferInputStream; import org.apache.hadoop.hbase.io.ByteBufferListOutputStream; import org.apache.hadoop.hbase.io.ByteBufferOutputStream; import org.apache.hadoop.hbase.io.ByteBufferPool; +import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; @@ -134,6 +141,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.htrace.TraceInfo; import org.codehaus.jackson.map.ObjectMapper; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService; import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream; import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream; @@ -423,6 +431,12 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { this.response = new BufferChain(responseBufs); } + protected synchronized void setConnectionHeaderResponse(ByteBuffer response) { + ByteBuffer[] responseBufs = new ByteBuffer[1]; + responseBufs[0] = response; + this.response = new BufferChain(responseBufs); + } + protected synchronized void setResponse(Object m, final CellScanner cells, Throwable t, String errorMsg) { if (this.isError) return; @@ -565,9 +579,16 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { byte [] responseBytes = bc.getBytes(); byte [] token; // synchronization may be needed since there can be multiple Handler - // threads using saslServer to wrap responses. - synchronized (connection.saslServer) { - token = connection.saslServer.wrap(responseBytes, 0, responseBytes.length); + // 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 (LOG.isTraceEnabled()) { LOG.trace("Adding saslServer wrapped token of size " + token.length @@ -1255,7 +1276,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { private ByteBuffer unwrappedDataLengthBuffer = ByteBuffer.allocate(4); boolean useSasl; SaslServer saslServer; + private CryptoAES cryptoAES; private boolean useWrap = false; + private boolean useCryptoAesWrap = false; // Fake 'call' for failed authorization response private static final int AUTHORIZATION_FAILED_CALLID = -1; private final Call authFailedCall = new Call(AUTHORIZATION_FAILED_CALLID, null, null, null, @@ -1266,6 +1289,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { private static final int SASL_CALLID = -33; private final Call saslCall = new Call(SASL_CALLID, null, null, null, null, null, this, null, 0, null, null, 0); + // Fake 'call' for connection header response + private static final int CONNECTION_HEADER_RESPONSE_CALLID = -34; + private final Call setConnectionHeaderResponseCall = new Call(CONNECTION_HEADER_RESPONSE_CALLID, + null, null, null, null, null, this, null, 0, null, null, 0); // was authentication allowed with a fallback to simple auth private boolean authenticatedWithFallback; @@ -1376,7 +1403,13 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { processOneRpc(saslToken); } else { byte[] b = saslToken.array(); - byte [] plaintextData = saslServer.unwrap(b, saslToken.position(), saslToken.limit()); + byte [] plaintextData; + if (useCryptoAesWrap) { + // unwrap with CryptoAES + plaintextData = cryptoAES.unwrap(b, saslToken.position(), saslToken.limit()); + } else { + plaintextData = saslServer.unwrap(b, saslToken.position(), saslToken.limit()); + } processUnwrappedData(plaintextData); } } else { @@ -1503,6 +1536,31 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { } } + /** + * Send the response for connection header + */ + private void doConnectionHeaderResponse(byte[] wrappedCipherMetaData) throws IOException { + ByteBufferOutputStream response = null; + DataOutputStream out = null; + try { + response = new ByteBufferOutputStream(wrappedCipherMetaData.length + 4); + out = new DataOutputStream(response); + out.writeInt(wrappedCipherMetaData.length); + out.write(wrappedCipherMetaData); + + setConnectionHeaderResponseCall.setConnectionHeaderResponse(response.getByteBuffer()); + setConnectionHeaderResponseCall.responder = responder; + setConnectionHeaderResponseCall.sendResponseIfReady(); + } finally { + if (out != null) { + out.close(); + } + if (response != null) { + response.close(); + } + } + } + private void disposeSasl() { if (saslServer != null) { try { @@ -1744,6 +1802,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { this.service = getService(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); UserGroupInformation protocolUser = createUser(connectionHeader); if (!useSasl) { ugi = protocolUser; @@ -1792,8 +1854,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { AUDITLOG.info("Connection from " + this.hostAddress + " port: " + this.remotePort + " with unknown version info"); } - - } /** @@ -1820,6 +1880,92 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { } } + /** + * Set up cipher for rpc encryption with Apache Commons Crypto + * @throws FatalConnectionException + */ + private void setupCryptoCipher(final ConnectionHeader header, + RPCProtos.ConnectionHeaderResponse.Builder chrBuilder) throws FatalConnectionException { + // If simple auth, return + if (saslServer == null) return; + // check if rpc encryption with Crypto AES + String qop = (String) saslServer.getNegotiatedProperty(Sasl.QOP); + boolean isEncryption = SaslUtil.QualityOfProtection.PRIVACY + .getSaslQop().equalsIgnoreCase(qop); + boolean isCryptoAesEncryption = isEncryption && 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; + // Negotiates AES based on complete saslServer. + // The Crypto metadata need to be encrypted and send to client. + Properties properties = new Properties(); + // the property for SecureRandomFactory + properties.setProperty(CryptoRandomFactory.CLASSES_KEY, + conf.get("hbase.crypto.sasl.encryption.aes.crypto.random", + "org.apache.commons.crypto.random.JavaCryptoRandom")); + // the property for cipher class + properties.setProperty(CryptoCipherFactory.CLASSES_KEY, + conf.get("hbase.rpc.crypto.encryption.aes.cipher.class", + "org.apache.commons.crypto.cipher.JceCipher")); + + int cipherKeyBits = conf.getInt( + "hbase.rpc.crypto.encryption.aes.cipher.keySizeBits", 128); + // generate key and iv + if (cipherKeyBits % 8 != 0) { + throw new IllegalArgumentException("The AES cipher key size in bits" + + " should be a multiple of byte"); + } + int len = cipherKeyBits / 8; + byte[] inKey = new byte[len]; + byte[] outKey = new byte[len]; + byte[] inIv = new byte[len]; + byte[] outIv = new byte[len]; + + try { + // generate the cipher meta data with SecureRandom + CryptoRandom secureRandom = CryptoRandomFactory.getCryptoRandom(properties); + secureRandom.nextBytes(inKey); + secureRandom.nextBytes(outKey); + secureRandom.nextBytes(inIv); + secureRandom.nextBytes(outIv); + + // 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); + } + } + + private void responseConnectionHeader(RPCProtos.ConnectionHeaderResponse.Builder chrBuilder) + throws FatalConnectionException { + // Response the connection header if Crypto AES is enabled + if (!chrBuilder.hasCryptoCipherMeta()) 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); + + doConnectionHeaderResponse(saslServer.wrap(unwrapped, 0, unwrapped.length)); + } catch (IOException ex) { + throw new UnsupportedCryptoException(ex.getMessage(), ex); + } + } + private void processUnwrappedData(byte[] inBuf) throws IOException, InterruptedException { ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(inBuf)); @@ -1858,7 +2004,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { } } - private void processOneRpc(ByteBuffer buf) throws IOException, InterruptedException { if (connectionHeaderRead) { processRequest(buf); @@ -1988,6 +2133,11 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { } } + private ByteString getByteString(byte[] bytes) { + // return singleton to reduce object allocation + return (bytes.length == 0) ? ByteString.EMPTY : ByteString.copyFrom(bytes); + } + private boolean authorizeConnection() throws IOException { try { // If auth method is DIGEST, the token was obtained by the 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 7bc97ca73e1..b7d6f8764fa 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 @@ -26,6 +26,7 @@ import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConf import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; +import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; @@ -96,8 +97,8 @@ public class TestSecureIPC { @Parameters(name = "{index}: rpcClientImpl={0}") public static Collection parameters() { - return Arrays.asList(new Object[] { BlockingRpcClient.class.getName() }, - new Object[] { NettyRpcClient.class.getName() }); + return Arrays.asList(new Object[]{BlockingRpcClient.class.getName()}, + new Object[]{NettyRpcClient.class.getName()}); } @Parameter @@ -192,6 +193,42 @@ public class TestSecureIPC { callRpcService(User.create(ugi)); } + /** + * Test sasl encryption with Crypto AES. + * @throws Exception + */ + @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. + * @throws Exception + */ + @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(); diff --git a/pom.xml b/pom.xml index 35b970849f4..e08f64e4802 100644 --- a/pom.xml +++ b/pom.xml @@ -1228,6 +1228,7 @@ 2.11.6 1.46 1.0.0-RC2 + 1.0.0 2.4 1.8 @@ -1806,6 +1807,17 @@ kerb-simplekdc ${kerby.version} + + org.apache.commons + commons-crypto + ${commons-crypto.version} + + + net.java.dev.jna + jna + + +