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.buffer.Unpooled;
22 import io.netty.channel.ChannelHandlerContext;
23 import io.netty.channel.ChannelOutboundHandlerAdapter;
24 import io.netty.channel.ChannelPromise;
25 import io.netty.channel.CoalescingBufferQueue;
26 import io.netty.util.ReferenceCountUtil;
27 import io.netty.util.concurrent.PromiseCombiner;
28
29 import java.io.IOException;
30
31 import javax.security.sasl.SaslClient;
32
33 import org.apache.hadoop.hbase.classification.InterfaceAudience;
34
35
36
37
38 @InterfaceAudience.Private
39 public class SaslWrapHandler extends ChannelOutboundHandlerAdapter {
40
41 private final SaslClient saslClient;
42
43 private CoalescingBufferQueue queue;
44
45 public SaslWrapHandler(SaslClient saslClient) {
46 this.saslClient = saslClient;
47 }
48
49 @Override
50 public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
51 queue = new CoalescingBufferQueue(ctx.channel());
52 }
53
54 @Override
55 public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
56 throws Exception {
57 if (msg instanceof ByteBuf) {
58 queue.add((ByteBuf) msg, promise);
59 } else {
60 ctx.write(msg, promise);
61 }
62 }
63
64 @Override
65 public void flush(ChannelHandlerContext ctx) throws Exception {
66 if (queue.isEmpty()) {
67 return;
68 }
69 ByteBuf buf = null;
70 try {
71 ChannelPromise promise = ctx.newPromise();
72 int readableBytes = queue.readableBytes();
73 buf = queue.remove(readableBytes, promise);
74 byte[] bytes = new byte[readableBytes];
75 buf.readBytes(bytes);
76 byte[] wrapperBytes = saslClient.wrap(bytes, 0, bytes.length);
77 ChannelPromise lenPromise = ctx.newPromise();
78 ctx.write(ctx.alloc().buffer(4).writeInt(wrapperBytes.length), lenPromise);
79 ChannelPromise contentPromise = ctx.newPromise();
80 ctx.write(Unpooled.wrappedBuffer(wrapperBytes), contentPromise);
81 PromiseCombiner combiner = new PromiseCombiner();
82 combiner.addAll(lenPromise, contentPromise);
83 combiner.finish(promise);
84 ctx.flush();
85 } finally {
86 if (buf != null) {
87 ReferenceCountUtil.safeRelease(buf);
88 }
89 }
90 }
91
92 @Override
93 public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
94 if (!queue.isEmpty()) {
95 queue.releaseAndFailAll(new IOException("Connection closed"));
96 }
97 ctx.close(promise);
98 }
99 }