001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.security;
019
020import org.apache.hadoop.conf.Configuration;
021import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;
022import org.apache.hadoop.hbase.util.NettyFutureUtils;
023import org.apache.yetus.audience.InterfaceAudience;
024
025import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
026import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
027import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;
028import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
029import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
030import org.apache.hbase.thirdparty.io.netty.util.concurrent.Promise;
031
032import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
033
034/**
035 * Implement logic to deal with the rpc connection header.
036 * @since 2.0.0
037 */
038@InterfaceAudience.Private
039public class NettyHBaseRpcConnectionHeaderHandler extends SimpleChannelInboundHandler<ByteBuf> {
040
041  private final Promise<Boolean> saslPromise;
042
043  private final Configuration conf;
044
045  private final ByteBuf connectionHeaderWithLength;
046
047  public NettyHBaseRpcConnectionHeaderHandler(Promise<Boolean> saslPromise, Configuration conf,
048    ByteBuf connectionHeaderWithLength) {
049    this.saslPromise = saslPromise;
050    this.conf = conf;
051    this.connectionHeaderWithLength = connectionHeaderWithLength;
052  }
053
054  @Override
055  protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception {
056    // read the ConnectionHeaderResponse from server
057    int len = msg.readInt();
058    byte[] buff = new byte[len];
059    msg.readBytes(buff);
060
061    RPCProtos.ConnectionHeaderResponse connectionHeaderResponse =
062      RPCProtos.ConnectionHeaderResponse.parseFrom(buff);
063
064    // Get the CryptoCipherMeta, update the HBaseSaslRpcClient for Crypto Cipher
065    if (connectionHeaderResponse.hasCryptoCipherMeta()) {
066      CryptoAES cryptoAES =
067        EncryptionUtil.createCryptoAES(connectionHeaderResponse.getCryptoCipherMeta(), conf);
068      // replace the Sasl handler with Crypto AES handler
069      setupCryptoAESHandler(ctx.pipeline(), cryptoAES);
070    }
071    NettyFutureUtils.consume(saslPromise.setSuccess(true));
072  }
073
074  @Override
075  public void handlerAdded(ChannelHandlerContext ctx) {
076    try {
077      // send the connection header to server first
078      NettyFutureUtils.safeWriteAndFlush(ctx, connectionHeaderWithLength.retainedDuplicate());
079    } catch (Exception e) {
080      // the exception thrown by handlerAdded will not be passed to the exceptionCaught below
081      // because netty will remove a handler if handlerAdded throws an exception.
082      exceptionCaught(ctx, e);
083    }
084  }
085
086  @Override
087  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
088    saslPromise.tryFailure(cause);
089  }
090
091  /**
092   * Remove handlers for sasl encryption and add handlers for Crypto AES encryption
093   */
094  private void setupCryptoAESHandler(ChannelPipeline p, CryptoAES cryptoAES) {
095    p.remove(SaslWrapHandler.class);
096    p.remove(SaslUnwrapHandler.class);
097    String lengthDecoder = p.context(LengthFieldBasedFrameDecoder.class).name();
098    p.addAfter(lengthDecoder, null, new CryptoAESUnwrapHandler(cryptoAES));
099    p.addAfter(lengthDecoder, null, new CryptoAESWrapHandler(cryptoAES));
100  }
101}