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.ipc; 019 020import java.nio.ByteBuffer; 021import org.apache.hadoop.hbase.ipc.ServerRpcConnection.PreambleResponse; 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.FixedLengthFrameDecoder; 030import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder; 031 032/** 033 * Handle connection preamble. 034 * @since 2.0.0` 035 */ 036@InterfaceAudience.Private 037class NettyRpcServerPreambleHandler extends SimpleChannelInboundHandler<ByteBuf> { 038 039 static final String DECODER_NAME = "preambleDecoder"; 040 041 private final NettyRpcServer rpcServer; 042 private final NettyServerRpcConnection conn; 043 private boolean processPreambleError; 044 045 public NettyRpcServerPreambleHandler(NettyRpcServer rpcServer, NettyServerRpcConnection conn) { 046 this.rpcServer = rpcServer; 047 this.conn = conn; 048 } 049 050 static FixedLengthFrameDecoder createDecoder() { 051 FixedLengthFrameDecoder preambleDecoder = new FixedLengthFrameDecoder(6); 052 preambleDecoder.setSingleDecode(true); 053 return preambleDecoder; 054 } 055 056 @Override 057 protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { 058 if (processPreambleError) { 059 // if we failed to process preamble, we will close the connection immediately, but it is 060 // possible that we have already received some bytes after the 'preamble' so when closing, the 061 // netty framework will still pass them here. So we set a flag here to just skip processing 062 // these broken messages. 063 return; 064 } 065 ByteBuffer buf = ByteBuffer.allocate(msg.readableBytes()); 066 msg.readBytes(buf); 067 buf.flip(); 068 PreambleResponse resp = conn.processPreamble(buf); 069 if (resp == PreambleResponse.CLOSE) { 070 processPreambleError = true; 071 conn.close(); 072 return; 073 } 074 if (resp == PreambleResponse.CONTINUE) { 075 // we use a single decode decoder, so here we need to replace it with a new one so it will 076 // decode a new preamble header again 077 ctx.pipeline().replace(DECODER_NAME, DECODER_NAME, createDecoder()); 078 return; 079 } 080 // resp == PreambleResponse.SUCCEED 081 ChannelPipeline p = ctx.pipeline(); 082 if (conn.useSasl) { 083 LengthFieldBasedFrameDecoder decoder = 084 new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4); 085 decoder.setSingleDecode(true); 086 p.addBefore(NettyRpcServerResponseEncoder.NAME, NettyHBaseSaslRpcServerHandler.DECODER_NAME, 087 decoder).addBefore(NettyRpcServerResponseEncoder.NAME, null, 088 new NettyHBaseSaslRpcServerHandler(rpcServer, conn)); 089 } else { 090 conn.setupHandler(); 091 } 092 // add first and then remove, so the single decode decoder will pass the remaining bytes to the 093 // handler above. 094 p.remove(this); 095 p.remove(DECODER_NAME); 096 } 097 098 @Override 099 public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { 100 NettyRpcServer.LOG.warn("Connection {}; caught unexpected downstream exception.", 101 ctx.channel().remoteAddress(), cause); 102 NettyFutureUtils.safeClose(ctx); 103 } 104}