1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.security;
19
20 import io.netty.buffer.ByteBuf;
21 import io.netty.channel.ChannelHandlerContext;
22 import io.netty.channel.SimpleChannelInboundHandler;
23 import io.netty.util.concurrent.Promise;
24
25 import java.io.IOException;
26 import java.security.PrivilegedExceptionAction;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.ipc.FallbackDisallowedException;
32 import org.apache.hadoop.security.UserGroupInformation;
33 import org.apache.hadoop.security.token.Token;
34 import org.apache.hadoop.security.token.TokenIdentifier;
35
36
37
38
39 @InterfaceAudience.Private
40 public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<ByteBuf> {
41
42 private static final Log LOG = LogFactory.getLog(NettyHBaseSaslRpcClientHandler.class);
43
44 private final Promise<Boolean> saslPromise;
45
46 private final UserGroupInformation ugi;
47
48 private final NettyHBaseSaslRpcClient saslRpcClient;
49
50
51
52
53
54 public NettyHBaseSaslRpcClientHandler(Promise<Boolean> saslPromise, UserGroupInformation ugi,
55 AuthMethod method, Token<? extends TokenIdentifier> token, String serverPrincipal,
56 boolean fallbackAllowed, String rpcProtection) throws IOException {
57 this.saslPromise = saslPromise;
58 this.ugi = ugi;
59 this.saslRpcClient = new NettyHBaseSaslRpcClient(method, token, serverPrincipal,
60 fallbackAllowed, rpcProtection);
61 }
62
63 private void writeResponse(ChannelHandlerContext ctx, byte[] response) {
64 LOG.trace("Will send token of size " + response.length + " from initSASLContext.");
65 ctx.writeAndFlush(
66 ctx.alloc().buffer(4 + response.length).writeInt(response.length).writeBytes(response));
67 }
68
69 private void tryComplete(ChannelHandlerContext ctx) {
70 if (!saslRpcClient.isComplete()) {
71 return;
72 }
73 saslRpcClient.setupSaslHandler(ctx.pipeline());
74 saslPromise.setSuccess(true);
75 }
76
77 @Override
78 public void handlerAdded(ChannelHandlerContext ctx) {
79 try {
80 byte[] initialResponse = ugi.doAs(new PrivilegedExceptionAction<byte[]>() {
81
82 @Override
83 public byte[] run() throws Exception {
84 return saslRpcClient.getInitialResponse();
85 }
86 });
87 if (initialResponse != null) {
88 writeResponse(ctx, initialResponse);
89 }
90 tryComplete(ctx);
91 } catch (Exception e) {
92
93
94 exceptionCaught(ctx, e);
95 }
96 }
97
98 @Override
99 protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception {
100 int len = msg.readInt();
101 if (len == SaslUtil.SWITCH_TO_SIMPLE_AUTH) {
102 saslRpcClient.dispose();
103 if (saslRpcClient.fallbackAllowed) {
104 saslPromise.trySuccess(false);
105 } else {
106 saslPromise.tryFailure(new FallbackDisallowedException());
107 }
108 return;
109 }
110 LOG.trace("Will read input token of size " + len + " for processing by initSASLContext");
111 final byte[] challenge = new byte[len];
112 msg.readBytes(challenge);
113 byte[] response = ugi.doAs(new PrivilegedExceptionAction<byte[]>() {
114
115 @Override
116 public byte[] run() throws Exception {
117 return saslRpcClient.evaluateChallenge(challenge);
118 }
119 });
120 if (response != null) {
121 writeResponse(ctx, response);
122 }
123 tryComplete(ctx);
124 }
125
126 @Override
127 public void channelInactive(ChannelHandlerContext ctx) throws Exception {
128 saslRpcClient.dispose();
129 saslPromise.tryFailure(new IOException("Connection closed"));
130 ctx.fireChannelInactive();
131 }
132
133 @Override
134 public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
135 saslRpcClient.dispose();
136 saslPromise.tryFailure(cause);
137 }
138 }