Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-27185 Rewrite NettyRpcServer to decode rpc request with netty h… #4624

Merged
merged 1 commit into from
Jul 27, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -223,9 +223,6 @@ private void saslNegotiate(final Channel ch) {
public void operationComplete(Future<Boolean> future) throws Exception {
if (future.isSuccess()) {
ChannelPipeline p = ch.pipeline();
p.remove(SaslChallengeDecoder.class);
p.remove(NettyHBaseSaslRpcClientHandler.class);

// check if negotiate with server for connection header is necessary
if (saslHandler.isNeedProcessConnectionHeader()) {
Promise<Boolean> connectionHeaderPromise = ch.eventLoop().newPromise();
Expand Down

This file was deleted.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;
import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import org.apache.hbase.thirdparty.io.netty.util.concurrent.Promise;

import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
Expand Down Expand Up @@ -92,10 +91,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
* Remove handlers for sasl encryption and add handlers for Crypto AES encryption
*/
private void setupCryptoAESHandler(ChannelPipeline p, CryptoAES cryptoAES) {
p.remove(SaslWrapHandler.class);
p.remove(SaslUnwrapHandler.class);
String lengthDecoder = p.context(LengthFieldBasedFrameDecoder.class).name();
p.addAfter(lengthDecoder, null, new CryptoAESUnwrapHandler(cryptoAES));
p.addAfter(lengthDecoder, null, new CryptoAESWrapHandler(cryptoAES));
p.replace(SaslWrapHandler.class, null, new SaslWrapHandler(cryptoAES::wrap));
p.replace(SaslUnwrapHandler.class, null, new SaslUnwrapHandler(cryptoAES::unwrap));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -52,9 +52,9 @@ public void setupSaslHandler(ChannelPipeline p) {
return;
}
// add wrap and unwrap handlers to pipeline.
p.addFirst(new SaslWrapHandler(saslClient),
p.addFirst(new SaslWrapHandler(saslClient::wrap),
new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4),
new SaslUnwrapHandler(saslClient));
new SaslUnwrapHandler(saslClient::unwrap));
}

public String getSaslQOP() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
import org.apache.hadoop.hbase.ipc.FallbackDisallowedException;
import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProvider;
import org.apache.hadoop.hbase.util.NettyFutureUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
Expand All @@ -33,6 +34,7 @@

import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;
import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hbase.thirdparty.io.netty.util.concurrent.Promise;

Expand Down Expand Up @@ -77,7 +79,7 @@ public NettyHBaseSaslRpcClientHandler(Promise<Boolean> saslPromise, UserGroupInf

private void writeResponse(ChannelHandlerContext ctx, byte[] response) {
LOG.trace("Sending token size={} from initSASLContext.", response.length);
ctx.writeAndFlush(
NettyFutureUtils.safeWriteAndFlush(ctx,
ctx.alloc().buffer(4 + response.length).writeInt(response.length).writeBytes(response));
}

Expand All @@ -90,8 +92,11 @@ private void tryComplete(ChannelHandlerContext ctx) {
if (LOG.isTraceEnabled()) {
LOG.trace("SASL negotiation for {} is complete", provider.getSaslAuthMethod().getName());
}

ChannelPipeline p = ctx.pipeline();
saslRpcClient.setupSaslHandler(ctx.pipeline());
p.remove(SaslChallengeDecoder.class);
p.remove(this);

setCryptoAESOption();

saslPromise.setSuccess(true);
Expand All @@ -110,6 +115,9 @@ public boolean isNeedProcessConnectionHeader() {

@Override
public void handlerAdded(ChannelHandlerContext ctx) {
// dispose the saslRpcClient when the channel is closed, since saslRpcClient is final, it is
// safe to reference it in lambda expr.
NettyFutureUtils.addListener(ctx.channel().closeFuture(), f -> saslRpcClient.dispose());
try {
byte[] initialResponse = ugi.doAs(new PrivilegedExceptionAction<byte[]>() {

Expand Down Expand Up @@ -170,14 +178,12 @@ public byte[] run() throws Exception {

@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
saslRpcClient.dispose();
saslPromise.tryFailure(new ConnectionClosedException("Connection closed"));
ctx.fireChannelInactive();
}

@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
saslRpcClient.dispose();
saslPromise.tryFailure(cause);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.security;

import javax.security.sasl.SaslClient;
import javax.security.sasl.SaslException;
import org.apache.yetus.audience.InterfaceAudience;

import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
Expand All @@ -32,22 +32,20 @@
@InterfaceAudience.Private
public class SaslUnwrapHandler extends SimpleChannelInboundHandler<ByteBuf> {

private final SaslClient saslClient;

public SaslUnwrapHandler(SaslClient saslClient) {
this.saslClient = saslClient;
public interface Unwrapper {
byte[] unwrap(byte[] incoming, int offset, int len) throws SaslException;
}

@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
SaslUtil.safeDispose(saslClient);
ctx.fireChannelInactive();
private final Unwrapper unwrapper;

public SaslUnwrapHandler(Unwrapper unwrapper) {
this.unwrapper = unwrapper;
}

@Override
protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception {
byte[] bytes = new byte[msg.readableBytes()];
msg.readBytes(bytes);
ctx.fireChannelRead(Unpooled.wrappedBuffer(saslClient.unwrap(bytes, 0, bytes.length)));
ctx.fireChannelRead(Unpooled.wrappedBuffer(unwrapper.unwrap(bytes, 0, bytes.length)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.security;

import javax.security.sasl.SaslClient;
import javax.security.sasl.SaslException;
import org.apache.yetus.audience.InterfaceAudience;

import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
Expand All @@ -30,17 +30,21 @@
@InterfaceAudience.Private
public class SaslWrapHandler extends MessageToByteEncoder<ByteBuf> {

private final SaslClient saslClient;
public interface Wrapper {
byte[] wrap(byte[] outgoing, int offset, int len) throws SaslException;
}

private final Wrapper wrapper;

public SaslWrapHandler(SaslClient saslClient) {
this.saslClient = saslClient;
public SaslWrapHandler(Wrapper wrapper) {
this.wrapper = wrapper;
}

@Override
protected void encode(ChannelHandlerContext ctx, ByteBuf msg, ByteBuf out) throws Exception {
byte[] bytes = new byte[msg.readableBytes()];
msg.readBytes(bytes);
byte[] wrapperBytes = saslClient.wrap(bytes, 0, bytes.length);
byte[] wrapperBytes = wrapper.wrap(bytes, 0, bytes.length);
out.ensureWritable(4 + wrapperBytes.length);
out.writeInt(wrapperBytes.length);
out.writeBytes(wrapperBytes);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;

import java.io.IOException;
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
import org.apache.hadoop.hbase.security.SaslStatus;
import org.apache.hadoop.hbase.security.SaslUnwrapHandler;
import org.apache.hadoop.hbase.security.SaslWrapHandler;
import org.apache.hadoop.hbase.util.NettyFutureUtils;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;
import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder;

/**
* Implement SASL negotiation logic for rpc server.
*/
class NettyHBaseSaslRpcServerHandler extends SimpleChannelInboundHandler<ByteBuf> {

private static final Logger LOG = LoggerFactory.getLogger(NettyHBaseSaslRpcServerHandler.class);

static final String DECODER_NAME = "SaslNegotiationDecoder";

private final NettyRpcServer rpcServer;

private final NettyServerRpcConnection conn;

NettyHBaseSaslRpcServerHandler(NettyRpcServer rpcServer, NettyServerRpcConnection conn) {
this.rpcServer = rpcServer;
this.conn = conn;
}

private void doResponse(ChannelHandlerContext ctx, SaslStatus status, Writable rv,
String errorClass, String error) throws IOException {
// In my testing, have noticed that sasl messages are usually
// in the ballpark of 100-200. That's why the initial capacity is 256.
ByteBuf resp = ctx.alloc().buffer(256);
try (ByteBufOutputStream out = new ByteBufOutputStream(resp)) {
out.writeInt(status.state); // write status
if (status == SaslStatus.SUCCESS) {
rv.write(out);
} else {
WritableUtils.writeString(out, errorClass);
WritableUtils.writeString(out, error);
}
}
NettyFutureUtils.safeWriteAndFlush(ctx, resp);
}

@Override
protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception {
LOG.debug("Read input token of size={} for processing by saslServer.evaluateResponse()",
msg.readableBytes());
HBaseSaslRpcServer saslServer = conn.getOrCreateSaslServer();
byte[] saslToken = new byte[msg.readableBytes()];
msg.readBytes(saslToken, 0, saslToken.length);
byte[] replyToken = saslServer.evaluateResponse(saslToken);
if (replyToken != null) {
LOG.debug("Will send token of size {} from saslServer.", replyToken.length);
doResponse(ctx, SaslStatus.SUCCESS, new BytesWritable(replyToken), null, null);
}
if (saslServer.isComplete()) {
conn.finishSaslNegotiation();
String qop = saslServer.getNegotiatedQop();
boolean useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
ChannelPipeline p = ctx.pipeline();
if (useWrap) {
p.addFirst(new SaslWrapHandler(saslServer::wrap));
p.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4),
new SaslUnwrapHandler(saslServer::unwrap));
}
conn.setupDecoder();
p.remove(this);
p.remove(DECODER_NAME);
}
}

@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
LOG.error("Error when doing SASL handshade, provider={}", conn.provider, cause);
Throwable sendToClient = HBaseSaslRpcServer.unwrap(cause);
doResponse(ctx, SaslStatus.ERROR, null, sendToClient.getClass().getName(),
sendToClient.getLocalizedMessage());
rpcServer.metrics.authenticationFailure();
String clientIP = this.toString();
// attempting user could be null
RpcServer.AUDITLOG.warn("{}{}: {}", RpcServer.AUTH_FAILED_FOR, clientIP,
conn.saslServer != null ? conn.saslServer.getAttemptingUser() : "Unknown");
NettyFutureUtils.safeClose(ctx);
}
}
Loading