1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.ipc;
21
22 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION;
23
24 import java.io.ByteArrayInputStream;
25 import java.io.ByteArrayOutputStream;
26 import java.io.DataOutputStream;
27 import java.io.IOException;
28 import java.lang.reflect.Method;
29 import java.net.BindException;
30 import java.net.InetAddress;
31 import java.net.InetSocketAddress;
32 import java.net.ServerSocket;
33 import java.net.Socket;
34 import java.net.SocketException;
35 import java.net.UnknownHostException;
36 import java.nio.ByteBuffer;
37 import java.nio.channels.CancelledKeyException;
38 import java.nio.channels.Channels;
39 import java.nio.channels.ClosedChannelException;
40 import java.nio.channels.ReadableByteChannel;
41 import java.nio.channels.SelectionKey;
42 import java.nio.channels.Selector;
43 import java.nio.channels.ServerSocketChannel;
44 import java.nio.channels.SocketChannel;
45 import java.nio.channels.WritableByteChannel;
46 import java.security.PrivilegedExceptionAction;
47 import java.util.ArrayList;
48 import java.util.Collections;
49 import java.util.Iterator;
50 import java.util.LinkedList;
51 import java.util.List;
52 import java.util.Map;
53 import java.util.Random;
54 import java.util.concurrent.BlockingQueue;
55 import java.util.concurrent.ConcurrentHashMap;
56 import java.util.concurrent.ExecutorService;
57 import java.util.concurrent.Executors;
58 import java.util.concurrent.LinkedBlockingQueue;
59 import java.util.concurrent.atomic.AtomicInteger;
60
61 import javax.security.sasl.Sasl;
62 import javax.security.sasl.SaslException;
63 import javax.security.sasl.SaslServer;
64
65 import org.apache.commons.logging.Log;
66 import org.apache.commons.logging.LogFactory;
67 import org.apache.hadoop.classification.InterfaceAudience;
68 import org.apache.hadoop.conf.Configuration;
69 import org.apache.hadoop.hbase.CellScanner;
70 import org.apache.hadoop.hbase.HConstants;
71 import org.apache.hadoop.hbase.IpcProtocol;
72 import org.apache.hadoop.hbase.codec.Codec;
73 import org.apache.hadoop.hbase.exceptions.CallerDisconnectedException;
74 import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
75 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
76 import org.apache.hadoop.hbase.exceptions.ServerNotRunningYetException;
77 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
78 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
79 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
80 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
81 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
82 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
83 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
84 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
85 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
86 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
87 import org.apache.hadoop.hbase.security.AuthMethod;
88 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHandler;
89 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler;
90 import org.apache.hadoop.hbase.security.SaslStatus;
91 import org.apache.hadoop.hbase.security.SaslUtil;
92 import org.apache.hadoop.hbase.security.User;
93 import org.apache.hadoop.hbase.util.Bytes;
94 import org.apache.hadoop.hbase.util.Pair;
95 import org.apache.hadoop.io.BytesWritable;
96 import org.apache.hadoop.io.IntWritable;
97 import org.apache.hadoop.io.Writable;
98 import org.apache.hadoop.io.WritableUtils;
99 import org.apache.hadoop.io.compress.CompressionCodec;
100 import org.apache.hadoop.ipc.Server;
101 import org.apache.hadoop.security.AccessControlException;
102 import org.apache.hadoop.security.UserGroupInformation;
103 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
104 import org.apache.hadoop.security.authorize.AuthorizationException;
105 import org.apache.hadoop.security.authorize.ProxyUsers;
106 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
107 import org.apache.hadoop.security.token.SecretManager;
108 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
109 import org.apache.hadoop.security.token.TokenIdentifier;
110 import org.apache.hadoop.util.StringUtils;
111 import org.cliffc.high_scale_lib.Counter;
112 import org.cloudera.htrace.Sampler;
113 import org.cloudera.htrace.Span;
114 import org.cloudera.htrace.Trace;
115 import org.cloudera.htrace.TraceInfo;
116 import org.cloudera.htrace.impl.NullSpan;
117
118 import com.google.common.base.Function;
119 import com.google.common.util.concurrent.ThreadFactoryBuilder;
120 import com.google.protobuf.CodedInputStream;
121 import com.google.protobuf.Message;
122 import com.google.protobuf.Message.Builder;
123 import com.google.protobuf.TextFormat;
124
125
126
127
128
129
130
131
132
133
134
135 @InterfaceAudience.Private
136 public abstract class HBaseServer implements RpcServer {
137 public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer");
138 private final boolean authorize;
139 protected boolean isSecurityEnabled;
140
141 public static final byte CURRENT_VERSION = 0;
142
143
144
145
146 private static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
147
148
149
150
151 private static final int DEFAULT_MAX_CALLQUEUE_SIZE =
152 1024 * 1024 * 1024;
153
154 static final int BUFFER_INITIAL_SIZE = 1024;
155
156 private static final String WARN_DELAYED_CALLS =
157 "hbase.ipc.warn.delayedrpc.number";
158
159 private static final int DEFAULT_WARN_DELAYED_CALLS = 1000;
160
161 private final int warnDelayedCalls;
162
163 private AtomicInteger delayedCalls;
164 private final IPCUtil ipcUtil;
165
166 private static final String AUTH_FAILED_FOR = "Auth failed for ";
167 private static final String AUTH_SUCCESSFUL_FOR = "Auth successful for ";
168 private static final Log AUDITLOG =
169 LogFactory.getLog("SecurityLogger."+Server.class.getName());
170 protected SecretManager<TokenIdentifier> secretManager;
171 protected ServiceAuthorizationManager authManager;
172
173 protected static final ThreadLocal<RpcServer> SERVER =
174 new ThreadLocal<RpcServer>();
175 private volatile boolean started = false;
176 private static final ReflectionCache methodCache = new ReflectionCache();
177
178 private static final Map<String, Class<? extends IpcProtocol>> PROTOCOL_CACHE =
179 new ConcurrentHashMap<String, Class<? extends IpcProtocol>>();
180
181 @SuppressWarnings("unchecked")
182 static Class<? extends IpcProtocol> getProtocolClass(
183 String protocolName, Configuration conf)
184 throws ClassNotFoundException {
185 Class<? extends IpcProtocol> protocol =
186 PROTOCOL_CACHE.get(protocolName);
187
188 if (protocol == null) {
189 protocol = (Class<? extends IpcProtocol>)
190 conf.getClassByName(protocolName);
191 PROTOCOL_CACHE.put(protocolName, protocol);
192 }
193 return protocol;
194 }
195
196
197
198
199
200
201
202 public static RpcServer get() {
203 return SERVER.get();
204 }
205
206
207
208
209 protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
210
211
212
213
214
215 public static InetAddress getRemoteIp() {
216 Call call = CurCall.get();
217 if (call != null) {
218 return call.connection.socket.getInetAddress();
219 }
220 return null;
221 }
222
223
224
225
226 public static String getRemoteAddress() {
227 Call call = CurCall.get();
228 if (call != null) {
229 return call.connection.getHostAddress();
230 }
231 return null;
232 }
233
234 protected String bindAddress;
235 protected int port;
236 private int handlerCount;
237 private int priorityHandlerCount;
238 private int readThreads;
239 protected int maxIdleTime;
240
241
242 protected int thresholdIdleConnections;
243
244
245
246 int maxConnectionsToNuke;
247
248
249
250 protected MetricsHBaseServer metrics;
251
252 protected Configuration conf;
253
254 private int maxQueueLength;
255 private int maxQueueSize;
256 protected int socketSendBufferSize;
257 protected final boolean tcpNoDelay;
258 protected final boolean tcpKeepAlive;
259 protected final long purgeTimeout;
260
261 volatile protected boolean running = true;
262 protected BlockingQueue<Call> callQueue;
263 protected final Counter callQueueSize = new Counter();
264 protected BlockingQueue<Call> priorityCallQueue;
265
266 protected int highPriorityLevel;
267
268 protected final List<Connection> connectionList =
269 Collections.synchronizedList(new LinkedList<Connection>());
270
271
272 private Listener listener = null;
273 protected Responder responder = null;
274 protected int numConnections = 0;
275 private Handler[] handlers = null;
276 private Handler[] priorityHandlers = null;
277
278 protected BlockingQueue<Call> replicationQueue;
279 private int numOfReplicationHandlers = 0;
280 private Handler[] replicationHandlers = null;
281
282 protected HBaseRPCErrorHandler errorHandler = null;
283
284
285
286
287
288
289
290
291
292
293
294 public static void bind(ServerSocket socket, InetSocketAddress address,
295 int backlog) throws IOException {
296 try {
297 socket.bind(address, backlog);
298 } catch (BindException e) {
299 BindException bindException =
300 new BindException("Problem binding to " + address + " : " +
301 e.getMessage());
302 bindException.initCause(e);
303 throw bindException;
304 } catch (SocketException e) {
305
306
307 if ("Unresolved address".equals(e.getMessage())) {
308 throw new UnknownHostException("Invalid hostname for server: " +
309 address.getHostName());
310 }
311 throw e;
312 }
313 }
314
315
316 protected class Call implements RpcCallContext {
317 protected int id;
318 protected Method method;
319 protected Message param;
320
321 protected CellScanner cellScanner;
322 protected Connection connection;
323 protected long timestamp;
324
325 protected ByteBuffer response;
326 protected boolean delayResponse;
327 protected Responder responder;
328 protected boolean delayReturnValue;
329
330 protected long size;
331 protected boolean isError;
332 protected TraceInfo tinfo;
333
334 public Call(int id, Method method, Message param, CellScanner cellScanner,
335 Connection connection, Responder responder, long size, TraceInfo tinfo) {
336 this.id = id;
337 this.method = method;
338 this.param = param;
339 this.cellScanner = cellScanner;
340 this.connection = connection;
341 this.timestamp = System.currentTimeMillis();
342 this.response = null;
343 this.delayResponse = false;
344 this.responder = responder;
345 this.isError = false;
346 this.size = size;
347 this.tinfo = tinfo;
348 }
349
350 @Override
351 public String toString() {
352 return "callId: " + this.id + " methodName: " +
353 ((this.method != null)? this.method.getName(): null) + " param: " +
354 (this.param != null? TextFormat.shortDebugString(this.param): "") +
355 " from " + connection.toString();
356 }
357
358 protected synchronized void setSaslTokenResponse(ByteBuffer response) {
359 this.response = response;
360 }
361
362 protected synchronized void setResponse(Object m, final CellScanner cells,
363 Throwable t, String errorMsg) {
364 if (this.isError) return;
365 if (t != null) this.isError = true;
366 ByteBufferOutputStream bbos = null;
367 try {
368 ResponseHeader.Builder headerBuilder = ResponseHeader.newBuilder();
369
370 Message result = (Message)m;
371
372 headerBuilder.setCallId(this.id);
373 if (t != null) {
374 ExceptionResponse.Builder exceptionBuilder = ExceptionResponse.newBuilder();
375 exceptionBuilder.setExceptionClassName(t.getClass().getName());
376 exceptionBuilder.setStackTrace(errorMsg);
377 exceptionBuilder.setDoNotRetry(t instanceof DoNotRetryIOException);
378 if (t instanceof RegionMovedException) {
379
380
381
382 RegionMovedException rme = (RegionMovedException)t;
383 exceptionBuilder.setHostname(rme.getHostname());
384 exceptionBuilder.setPort(rme.getPort());
385 }
386
387 headerBuilder.setException(exceptionBuilder.build());
388 }
389 ByteBuffer cellBlock =
390 ipcUtil.buildCellBlock(this.connection.codec, this.connection.compressionCodec, cells);
391 if (cellBlock != null) {
392 CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
393
394 cellBlockBuilder.setLength(cellBlock.limit());
395 headerBuilder.setCellBlockMeta(cellBlockBuilder.build());
396 }
397 Message header = headerBuilder.build();
398 bbos = IPCUtil.write(header, result, cellBlock);
399 if (connection.useWrap) {
400 wrapWithSasl(bbos);
401 }
402 if (LOG.isDebugEnabled()) {
403 LOG.debug("Header " + TextFormat.shortDebugString(header) +
404 ", result " + (result != null? TextFormat.shortDebugString(result): "null"));
405 }
406 } catch (IOException e) {
407 LOG.warn("Exception while creating response " + e);
408 }
409 ByteBuffer bb = null;
410 if (bbos != null) {
411
412 bb = bbos.getByteBuffer();
413 bb.position(0);
414 }
415 this.response = bb;
416 }
417
418 private void wrapWithSasl(ByteBufferOutputStream response)
419 throws IOException {
420 if (connection.useSasl) {
421
422 ByteBuffer buf = response.getByteBuffer();
423 byte[] token;
424
425
426 synchronized (connection.saslServer) {
427 token = connection.saslServer.wrap(buf.array(),
428 buf.arrayOffset(), buf.remaining());
429 }
430 if (LOG.isDebugEnabled())
431 LOG.debug("Adding saslServer wrapped token of size " + token.length
432 + " as call response.");
433 buf.clear();
434 DataOutputStream saslOut = new DataOutputStream(response);
435 saslOut.writeInt(token.length);
436 saslOut.write(token, 0, token.length);
437 }
438 }
439
440 @Override
441 public synchronized void endDelay(Object result) throws IOException {
442 assert this.delayResponse;
443 assert this.delayReturnValue || result == null;
444 this.delayResponse = false;
445 delayedCalls.decrementAndGet();
446 if (this.delayReturnValue) {
447 this.setResponse(result, null, null, null);
448 }
449 this.responder.doRespond(this);
450 }
451
452 @Override
453 public synchronized void endDelay() throws IOException {
454 this.endDelay(null);
455 }
456
457 @Override
458 public synchronized void startDelay(boolean delayReturnValue) {
459 assert !this.delayResponse;
460 this.delayResponse = true;
461 this.delayReturnValue = delayReturnValue;
462 int numDelayed = delayedCalls.incrementAndGet();
463 if (numDelayed > warnDelayedCalls) {
464 LOG.warn("Too many delayed calls: limit " + warnDelayedCalls +
465 " current " + numDelayed);
466 }
467 }
468
469 @Override
470 public synchronized void endDelayThrowing(Throwable t) throws IOException {
471 this.setResponse(null, null, t, StringUtils.stringifyException(t));
472 this.delayResponse = false;
473 this.sendResponseIfReady();
474 }
475
476 @Override
477 public synchronized boolean isDelayed() {
478 return this.delayResponse;
479 }
480
481 @Override
482 public synchronized boolean isReturnValueDelayed() {
483 return this.delayReturnValue;
484 }
485
486 @Override
487 public void throwExceptionIfCallerDisconnected() throws CallerDisconnectedException {
488 if (!connection.channel.isOpen()) {
489 long afterTime = System.currentTimeMillis() - timestamp;
490 throw new CallerDisconnectedException(
491 "Aborting call " + this + " after " + afterTime + " ms, since " +
492 "caller disconnected");
493 }
494 }
495
496 public long getSize() {
497 return this.size;
498 }
499
500
501
502
503
504
505 public synchronized void sendResponseIfReady() throws IOException {
506 if (!this.delayResponse) {
507 this.responder.doRespond(this);
508 }
509 }
510 }
511
512
513 private class Listener extends Thread {
514
515 private ServerSocketChannel acceptChannel = null;
516 private Selector selector = null;
517 private Reader[] readers = null;
518 private int currentReader = 0;
519 private InetSocketAddress address;
520 private Random rand = new Random();
521 private long lastCleanupRunTime = 0;
522
523 private long cleanupInterval = 10000;
524
525 private int backlogLength = conf.getInt("ipc.server.listen.queue.size", 128);
526
527 private ExecutorService readPool;
528
529 public Listener() throws IOException {
530 address = new InetSocketAddress(bindAddress, port);
531
532 acceptChannel = ServerSocketChannel.open();
533 acceptChannel.configureBlocking(false);
534
535
536 bind(acceptChannel.socket(), address, backlogLength);
537 port = acceptChannel.socket().getLocalPort();
538
539 selector= Selector.open();
540
541 readers = new Reader[readThreads];
542 readPool = Executors.newFixedThreadPool(readThreads,
543 new ThreadFactoryBuilder().setNameFormat(
544 "IPC Reader %d on port " + port).setDaemon(true).build());
545 for (int i = 0; i < readThreads; ++i) {
546 Reader reader = new Reader();
547 readers[i] = reader;
548 readPool.execute(reader);
549 }
550 LOG.info(getName() + ": started " + readThreads + " reader(s) in Listener.");
551
552
553 acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
554 this.setName("IPC Server listener on " + port);
555 this.setDaemon(true);
556 }
557
558
559 private class Reader implements Runnable {
560 private volatile boolean adding = false;
561 private final Selector readSelector;
562
563 Reader() throws IOException {
564 this.readSelector = Selector.open();
565 }
566 public void run() {
567 try {
568 doRunLoop();
569 } finally {
570 try {
571 readSelector.close();
572 } catch (IOException ioe) {
573 LOG.error(getName() + ": error closing read selector in " + getName(), ioe);
574 }
575 }
576 }
577
578 private synchronized void doRunLoop() {
579 while (running) {
580 SelectionKey key = null;
581 try {
582 readSelector.select();
583 while (adding) {
584 this.wait(1000);
585 }
586
587 Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
588 while (iter.hasNext()) {
589 key = iter.next();
590 iter.remove();
591 if (key.isValid()) {
592 if (key.isReadable()) {
593 doRead(key);
594 }
595 }
596 key = null;
597 }
598 } catch (InterruptedException e) {
599 if (running) {
600 LOG.info(getName() + ": unexpectedly interrupted: " +
601 StringUtils.stringifyException(e));
602 }
603 } catch (IOException ex) {
604 LOG.error(getName() + ": error in Reader", ex);
605 }
606 }
607 }
608
609
610
611
612
613
614
615
616 public void startAdd() {
617 adding = true;
618 readSelector.wakeup();
619 }
620
621 public synchronized SelectionKey registerChannel(SocketChannel channel)
622 throws IOException {
623 return channel.register(readSelector, SelectionKey.OP_READ);
624 }
625
626 public synchronized void finishAdd() {
627 adding = false;
628 this.notify();
629 }
630 }
631
632
633
634
635
636
637
638
639 private void cleanupConnections(boolean force) {
640 if (force || numConnections > thresholdIdleConnections) {
641 long currentTime = System.currentTimeMillis();
642 if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
643 return;
644 }
645 int start = 0;
646 int end = numConnections - 1;
647 if (!force) {
648 start = rand.nextInt() % numConnections;
649 end = rand.nextInt() % numConnections;
650 int temp;
651 if (end < start) {
652 temp = start;
653 start = end;
654 end = temp;
655 }
656 }
657 int i = start;
658 int numNuked = 0;
659 while (i <= end) {
660 Connection c;
661 synchronized (connectionList) {
662 try {
663 c = connectionList.get(i);
664 } catch (Exception e) {return;}
665 }
666 if (c.timedOut(currentTime)) {
667 if (LOG.isDebugEnabled())
668 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
669 closeConnection(c);
670 numNuked++;
671 end--;
672
673 c = null;
674 if (!force && numNuked == maxConnectionsToNuke) break;
675 }
676 else i++;
677 }
678 lastCleanupRunTime = System.currentTimeMillis();
679 }
680 }
681
682 @Override
683 public void run() {
684 LOG.info(getName() + ": starting");
685 SERVER.set(HBaseServer.this);
686
687 while (running) {
688 SelectionKey key = null;
689 try {
690 selector.select();
691 Iterator<SelectionKey> iter = selector.selectedKeys().iterator();
692 while (iter.hasNext()) {
693 key = iter.next();
694 iter.remove();
695 try {
696 if (key.isValid()) {
697 if (key.isAcceptable())
698 doAccept(key);
699 }
700 } catch (IOException ignored) {
701 }
702 key = null;
703 }
704 } catch (OutOfMemoryError e) {
705 if (errorHandler != null) {
706 if (errorHandler.checkOOME(e)) {
707 LOG.info(getName() + ": exiting on OutOfMemoryError");
708 closeCurrentConnection(key, e);
709 cleanupConnections(true);
710 return;
711 }
712 } else {
713
714
715
716 LOG.warn(getName() + ": OutOfMemoryError in server select", e);
717 closeCurrentConnection(key, e);
718 cleanupConnections(true);
719 try { Thread.sleep(60000); } catch (Exception ignored) {}
720 }
721 } catch (Exception e) {
722 closeCurrentConnection(key, e);
723 }
724 cleanupConnections(false);
725 }
726 LOG.info(getName() + ": stopping");
727
728 synchronized (this) {
729 try {
730 acceptChannel.close();
731 selector.close();
732 } catch (IOException ignored) { }
733
734 selector= null;
735 acceptChannel= null;
736
737
738 while (!connectionList.isEmpty()) {
739 closeConnection(connectionList.remove(0));
740 }
741 }
742 }
743
744 private void closeCurrentConnection(SelectionKey key, Throwable e) {
745 if (key != null) {
746 Connection c = (Connection)key.attachment();
747 if (c != null) {
748 if (LOG.isDebugEnabled()) {
749 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
750 (e != null ? " on error " + e.getMessage() : ""));
751 }
752 closeConnection(c);
753 key.attach(null);
754 }
755 }
756 }
757
758 InetSocketAddress getAddress() {
759 return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
760 }
761
762 void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
763 Connection c;
764 ServerSocketChannel server = (ServerSocketChannel) key.channel();
765
766 SocketChannel channel;
767 while ((channel = server.accept()) != null) {
768 channel.configureBlocking(false);
769 channel.socket().setTcpNoDelay(tcpNoDelay);
770 channel.socket().setKeepAlive(tcpKeepAlive);
771
772 Reader reader = getReader();
773 try {
774 reader.startAdd();
775 SelectionKey readKey = reader.registerChannel(channel);
776 c = getConnection(channel, System.currentTimeMillis());
777 readKey.attach(c);
778 synchronized (connectionList) {
779 connectionList.add(numConnections, c);
780 numConnections++;
781 }
782 if (LOG.isDebugEnabled())
783 LOG.debug(getName() + ": connection from " + c.toString() +
784 "; # active connections: " + numConnections +
785 "; # queued calls: " + callQueue.size());
786 } finally {
787 reader.finishAdd();
788 }
789 }
790 }
791
792 void doRead(SelectionKey key) throws InterruptedException {
793 int count = 0;
794 Connection c = (Connection)key.attachment();
795 if (c == null) {
796 return;
797 }
798 c.setLastContact(System.currentTimeMillis());
799 try {
800 count = c.readAndProcess();
801 } catch (InterruptedException ieo) {
802 throw ieo;
803 } catch (Exception e) {
804 LOG.warn(getName() + ": count of bytes read: " + count, e);
805 count = -1;
806 }
807 if (count < 0) {
808 if (LOG.isDebugEnabled()) {
809 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
810 ", because count=" + count +
811 ". Number of active connections: " + numConnections);
812 }
813 closeConnection(c);
814
815 } else {
816 c.setLastContact(System.currentTimeMillis());
817 }
818 }
819
820 synchronized void doStop() {
821 if (selector != null) {
822 selector.wakeup();
823 Thread.yield();
824 }
825 if (acceptChannel != null) {
826 try {
827 acceptChannel.socket().close();
828 } catch (IOException e) {
829 LOG.info(getName() + ": exception in closing listener socket. " + e);
830 }
831 }
832 readPool.shutdownNow();
833 }
834
835
836
837 Reader getReader() {
838 currentReader = (currentReader + 1) % readers.length;
839 return readers[currentReader];
840 }
841 }
842
843
844 protected class Responder extends Thread {
845 private final Selector writeSelector;
846 private int pending;
847
848 Responder() throws IOException {
849 this.setName("IPC Server Responder");
850 this.setDaemon(true);
851 writeSelector = Selector.open();
852 pending = 0;
853 }
854
855 @Override
856 public void run() {
857 LOG.info(getName() + ": starting");
858 SERVER.set(HBaseServer.this);
859 try {
860 doRunLoop();
861 } finally {
862 LOG.info(getName() + ": stopping");
863 try {
864 writeSelector.close();
865 } catch (IOException ioe) {
866 LOG.error(getName() + ": couldn't close write selector", ioe);
867 }
868 }
869 }
870
871 private void doRunLoop() {
872 long lastPurgeTime = 0;
873
874 while (running) {
875 try {
876 waitPending();
877 writeSelector.select(purgeTimeout);
878 Iterator<SelectionKey> iter = writeSelector.selectedKeys().iterator();
879 while (iter.hasNext()) {
880 SelectionKey key = iter.next();
881 iter.remove();
882 try {
883 if (key.isValid() && key.isWritable()) {
884 doAsyncWrite(key);
885 }
886 } catch (IOException e) {
887 LOG.info(getName() + ": asyncWrite", e);
888 }
889 }
890 long now = System.currentTimeMillis();
891 if (now < lastPurgeTime + purgeTimeout) {
892 continue;
893 }
894 lastPurgeTime = now;
895
896
897
898
899 if (LOG.isDebugEnabled()) LOG.debug(getName() + ": checking for old call responses.");
900 ArrayList<Call> calls;
901
902
903 synchronized (writeSelector.keys()) {
904 calls = new ArrayList<Call>(writeSelector.keys().size());
905 iter = writeSelector.keys().iterator();
906 while (iter.hasNext()) {
907 SelectionKey key = iter.next();
908 Call call = (Call)key.attachment();
909 if (call != null && key.channel() == call.connection.channel) {
910 calls.add(call);
911 }
912 }
913 }
914
915 for(Call call : calls) {
916 try {
917 doPurge(call, now);
918 } catch (IOException e) {
919 LOG.warn(getName() + ": error in purging old calls " + e);
920 }
921 }
922 } catch (OutOfMemoryError e) {
923 if (errorHandler != null) {
924 if (errorHandler.checkOOME(e)) {
925 LOG.info(getName() + ": exiting on OutOfMemoryError");
926 return;
927 }
928 } else {
929
930
931
932
933
934 LOG.warn(getName() + ": OutOfMemoryError in server select", e);
935 try { Thread.sleep(60000); } catch (Exception ignored) {}
936 }
937 } catch (Exception e) {
938 LOG.warn(getName() + ": exception in Responder " +
939 StringUtils.stringifyException(e));
940 }
941 }
942 LOG.info(getName() + ": stopped");
943 }
944
945 private void doAsyncWrite(SelectionKey key) throws IOException {
946 Call call = (Call)key.attachment();
947 if (call == null) {
948 return;
949 }
950 if (key.channel() != call.connection.channel) {
951 throw new IOException("doAsyncWrite: bad channel");
952 }
953
954 synchronized(call.connection.responseQueue) {
955 if (processResponse(call.connection.responseQueue, false)) {
956 try {
957 key.interestOps(0);
958 } catch (CancelledKeyException e) {
959
960
961
962
963
964 LOG.warn("Exception while changing ops : " + e);
965 }
966 }
967 }
968 }
969
970
971
972
973
974 private void doPurge(Call call, long now) throws IOException {
975 synchronized (call.connection.responseQueue) {
976 Iterator<Call> iter = call.connection.responseQueue.listIterator(0);
977 while (iter.hasNext()) {
978 Call nextCall = iter.next();
979 if (now > nextCall.timestamp + purgeTimeout) {
980 closeConnection(nextCall.connection);
981 break;
982 }
983 }
984 }
985 }
986
987
988
989
990 private boolean processResponse(final LinkedList<Call> responseQueue, boolean inHandler)
991 throws IOException {
992 boolean error = true;
993 boolean done = false;
994 int numElements;
995 Call call = null;
996 try {
997
998 synchronized (responseQueue) {
999
1000
1001
1002 numElements = responseQueue.size();
1003 if (numElements == 0) {
1004 error = false;
1005 return true;
1006 }
1007
1008
1009
1010 call = responseQueue.removeFirst();
1011 SocketChannel channel = call.connection.channel;
1012
1013
1014
1015 int numBytes = channelWrite(channel, call.response);
1016 if (numBytes < 0) {
1017 return true;
1018 }
1019 if (!call.response.hasRemaining()) {
1020 call.connection.decRpcCount();
1021
1022 if (numElements == 1) {
1023 done = true;
1024 } else {
1025 done = false;
1026 }
1027 if (LOG.isDebugEnabled()) {
1028 LOG.debug(getName() + ": callId: " + call.id + " sent, wrote " + numBytes +
1029 " bytes.");
1030 }
1031 } else {
1032
1033
1034
1035
1036 call.connection.responseQueue.addFirst(call);
1037
1038 if (inHandler) {
1039
1040 call.timestamp = System.currentTimeMillis();
1041 if (enqueueInSelector(call))
1042 done = true;
1043 }
1044 if (LOG.isDebugEnabled()) {
1045 LOG.debug(getName() + call.toString() + " partially sent, wrote " +
1046 numBytes + " bytes.");
1047 }
1048 }
1049 error = false;
1050 }
1051 } finally {
1052 if (error && call != null) {
1053 LOG.warn(getName() + call.toString() + ": output error");
1054 done = true;
1055 closeConnection(call.connection);
1056 }
1057 }
1058 return done;
1059 }
1060
1061
1062
1063
1064 private boolean enqueueInSelector(Call call) throws IOException {
1065 boolean done = false;
1066 incPending();
1067 try {
1068
1069
1070 SocketChannel channel = call.connection.channel;
1071 writeSelector.wakeup();
1072 channel.register(writeSelector, SelectionKey.OP_WRITE, call);
1073 } catch (ClosedChannelException e) {
1074
1075 done = true;
1076 } finally {
1077 decPending();
1078 }
1079 return done;
1080 }
1081
1082
1083
1084
1085 void doRespond(Call call) throws IOException {
1086
1087 call.timestamp = System.currentTimeMillis();
1088
1089 boolean doRegister = false;
1090 synchronized (call.connection.responseQueue) {
1091 call.connection.responseQueue.addLast(call);
1092 if (call.connection.responseQueue.size() == 1) {
1093 doRegister = !processResponse(call.connection.responseQueue, false);
1094 }
1095 }
1096 if (doRegister) {
1097 enqueueInSelector(call);
1098 }
1099 }
1100
1101 private synchronized void incPending() {
1102 pending++;
1103 }
1104
1105 private synchronized void decPending() {
1106 pending--;
1107 notify();
1108 }
1109
1110 private synchronized void waitPending() throws InterruptedException {
1111 while (pending > 0) {
1112 wait();
1113 }
1114 }
1115 }
1116
1117 @SuppressWarnings("serial")
1118 public static class CallQueueTooBigException extends IOException {
1119 CallQueueTooBigException() {
1120 super();
1121 }
1122 }
1123
1124 private Function<Pair<RequestHeader, Message>, Integer> qosFunction = null;
1125
1126
1127
1128
1129
1130
1131
1132 @Override
1133 public void setQosFunction(Function<Pair<RequestHeader, Message>, Integer> newFunc) {
1134 qosFunction = newFunc;
1135 }
1136
1137 protected int getQosLevel(Pair<RequestHeader, Message> headerAndParam) {
1138 if (qosFunction == null) return 0;
1139 Integer res = qosFunction.apply(headerAndParam);
1140 return res == null? 0: res;
1141 }
1142
1143
1144 public class Connection {
1145
1146 private boolean connectionPreambleRead = false;
1147
1148 private boolean connectionHeaderRead = false;
1149 protected SocketChannel channel;
1150 private ByteBuffer data;
1151 private ByteBuffer dataLengthBuffer;
1152 protected final LinkedList<Call> responseQueue;
1153 private volatile int rpcCount = 0;
1154 private long lastContact;
1155 private InetAddress addr;
1156 protected Socket socket;
1157
1158
1159 protected String hostAddress;
1160 protected int remotePort;
1161 ConnectionHeader connectionHeader;
1162
1163
1164
1165 private Codec codec;
1166
1167
1168
1169 private CompressionCodec compressionCodec;
1170 Class<? extends IpcProtocol> protocol;
1171 protected UserGroupInformation user = null;
1172 private AuthMethod authMethod;
1173 private boolean saslContextEstablished;
1174 private boolean skipInitialSaslHandshake;
1175 private ByteBuffer unwrappedData;
1176
1177 private ByteBuffer unwrappedDataLengthBuffer;
1178 boolean useSasl;
1179 SaslServer saslServer;
1180 private boolean useWrap = false;
1181
1182 private static final int AUTHROIZATION_FAILED_CALLID = -1;
1183 private final Call authFailedCall =
1184 new Call(AUTHROIZATION_FAILED_CALLID, null, null, null, this, null, 0, null);
1185 private ByteArrayOutputStream authFailedResponse =
1186 new ByteArrayOutputStream();
1187
1188 private static final int SASL_CALLID = -33;
1189 private final Call saslCall =
1190 new Call(SASL_CALLID, null, null, null, this, null, 0, null);
1191
1192 public UserGroupInformation attemptingUser = null;
1193
1194 public Connection(SocketChannel channel, long lastContact) {
1195 this.channel = channel;
1196 this.lastContact = lastContact;
1197 this.data = null;
1198 this.dataLengthBuffer = ByteBuffer.allocate(4);
1199 this.socket = channel.socket();
1200 this.addr = socket.getInetAddress();
1201 if (addr == null) {
1202 this.hostAddress = "*Unknown*";
1203 } else {
1204 this.hostAddress = addr.getHostAddress();
1205 }
1206 this.remotePort = socket.getPort();
1207 this.responseQueue = new LinkedList<Call>();
1208 if (socketSendBufferSize != 0) {
1209 try {
1210 socket.setSendBufferSize(socketSendBufferSize);
1211 } catch (IOException e) {
1212 LOG.warn("Connection: unable to set socket send buffer size to " +
1213 socketSendBufferSize);
1214 }
1215 }
1216 }
1217
1218 @Override
1219 public String toString() {
1220 return getHostAddress() + ":" + remotePort;
1221 }
1222
1223 public String getHostAddress() {
1224 return hostAddress;
1225 }
1226
1227 public InetAddress getHostInetAddress() {
1228 return addr;
1229 }
1230
1231 public int getRemotePort() {
1232 return remotePort;
1233 }
1234
1235 public void setLastContact(long lastContact) {
1236 this.lastContact = lastContact;
1237 }
1238
1239 public long getLastContact() {
1240 return lastContact;
1241 }
1242
1243
1244 private boolean isIdle() {
1245 return rpcCount == 0;
1246 }
1247
1248
1249 protected void decRpcCount() {
1250 rpcCount--;
1251 }
1252
1253
1254 protected void incRpcCount() {
1255 rpcCount++;
1256 }
1257
1258 protected boolean timedOut(long currentTime) {
1259 return isIdle() && currentTime - lastContact > maxIdleTime;
1260 }
1261
1262 private UserGroupInformation getAuthorizedUgi(String authorizedId)
1263 throws IOException {
1264 if (authMethod == AuthMethod.DIGEST) {
1265 TokenIdentifier tokenId = HBaseSaslRpcServer.getIdentifier(authorizedId,
1266 secretManager);
1267 UserGroupInformation ugi = tokenId.getUser();
1268 if (ugi == null) {
1269 throw new AccessControlException(
1270 "Can't retrieve username from tokenIdentifier.");
1271 }
1272 ugi.addTokenIdentifier(tokenId);
1273 return ugi;
1274 } else {
1275 return UserGroupInformation.createRemoteUser(authorizedId);
1276 }
1277 }
1278
1279 private void saslReadAndProcess(byte[] saslToken) throws IOException,
1280 InterruptedException {
1281 if (saslContextEstablished) {
1282 if (LOG.isDebugEnabled())
1283 LOG.debug("Have read input token of size " + saslToken.length
1284 + " for processing by saslServer.unwrap()");
1285
1286 if (!useWrap) {
1287 processOneRpc(saslToken);
1288 } else {
1289 byte[] plaintextData = saslServer.unwrap(saslToken, 0,
1290 saslToken.length);
1291 processUnwrappedData(plaintextData);
1292 }
1293 } else {
1294 byte[] replyToken = null;
1295 try {
1296 if (saslServer == null) {
1297 switch (authMethod) {
1298 case DIGEST:
1299 if (secretManager == null) {
1300 throw new AccessControlException(
1301 "Server is not configured to do DIGEST authentication.");
1302 }
1303 saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
1304 .getMechanismName(), null, SaslUtil.SASL_DEFAULT_REALM,
1305 SaslUtil.SASL_PROPS, new SaslDigestCallbackHandler(
1306 secretManager, this));
1307 break;
1308 default:
1309 UserGroupInformation current = UserGroupInformation
1310 .getCurrentUser();
1311 String fullName = current.getUserName();
1312 if (LOG.isDebugEnabled()) {
1313 LOG.debug("Kerberos principal name is " + fullName);
1314 }
1315 final String names[] = SaslUtil.splitKerberosName(fullName);
1316 if (names.length != 3) {
1317 throw new AccessControlException(
1318 "Kerberos principal name does NOT have the expected "
1319 + "hostname part: " + fullName);
1320 }
1321 current.doAs(new PrivilegedExceptionAction<Object>() {
1322 @Override
1323 public Object run() throws SaslException {
1324 saslServer = Sasl.createSaslServer(AuthMethod.KERBEROS
1325 .getMechanismName(), names[0], names[1],
1326 SaslUtil.SASL_PROPS, new SaslGssCallbackHandler());
1327 return null;
1328 }
1329 });
1330 }
1331 if (saslServer == null)
1332 throw new AccessControlException(
1333 "Unable to find SASL server implementation for "
1334 + authMethod.getMechanismName());
1335 if (LOG.isDebugEnabled()) {
1336 LOG.debug("Created SASL server with mechanism = " + authMethod.getMechanismName());
1337 }
1338 }
1339 if (LOG.isDebugEnabled()) {
1340 LOG.debug("Have read input token of size " + saslToken.length
1341 + " for processing by saslServer.evaluateResponse()");
1342 }
1343 replyToken = saslServer.evaluateResponse(saslToken);
1344 } catch (IOException e) {
1345 IOException sendToClient = e;
1346 Throwable cause = e;
1347 while (cause != null) {
1348 if (cause instanceof InvalidToken) {
1349 sendToClient = (InvalidToken) cause;
1350 break;
1351 }
1352 cause = cause.getCause();
1353 }
1354 doRawSaslReply(SaslStatus.ERROR, null, sendToClient.getClass().getName(),
1355 sendToClient.getLocalizedMessage());
1356 metrics.authenticationFailure();
1357 String clientIP = this.toString();
1358
1359 AUDITLOG.warn(AUTH_FAILED_FOR + clientIP + ":" + attemptingUser);
1360 throw e;
1361 }
1362 if (replyToken != null) {
1363 if (LOG.isDebugEnabled()) {
1364 LOG.debug("Will send token of size " + replyToken.length
1365 + " from saslServer.");
1366 }
1367 doRawSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null,
1368 null);
1369 }
1370 if (saslServer.isComplete()) {
1371 String qop = (String) saslServer.getNegotiatedProperty(Sasl.QOP);
1372 useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
1373 user = getAuthorizedUgi(saslServer.getAuthorizationID());
1374 if (LOG.isDebugEnabled()) {
1375 LOG.debug("SASL server context established. Authenticated client: "
1376 + user + ". Negotiated QoP is "
1377 + saslServer.getNegotiatedProperty(Sasl.QOP));
1378 }
1379 metrics.authenticationSuccess();
1380 AUDITLOG.info(AUTH_SUCCESSFUL_FOR + user);
1381 saslContextEstablished = true;
1382 }
1383 }
1384 }
1385
1386
1387
1388 private void doRawSaslReply(SaslStatus status, Writable rv,
1389 String errorClass, String error) throws IOException {
1390
1391
1392 ByteBufferOutputStream saslResponse = new ByteBufferOutputStream(256);
1393 DataOutputStream out = new DataOutputStream(saslResponse);
1394 out.writeInt(status.state);
1395 if (status == SaslStatus.SUCCESS) {
1396 rv.write(out);
1397 } else {
1398 WritableUtils.writeString(out, errorClass);
1399 WritableUtils.writeString(out, error);
1400 }
1401 saslCall.setSaslTokenResponse(saslResponse.getByteBuffer());
1402 saslCall.responder = responder;
1403 saslCall.sendResponseIfReady();
1404 }
1405
1406 private void disposeSasl() {
1407 if (saslServer != null) {
1408 try {
1409 saslServer.dispose();
1410 saslServer = null;
1411 } catch (SaslException ignored) {
1412 }
1413 }
1414 }
1415
1416
1417
1418
1419
1420
1421
1422
1423 public int readAndProcess() throws IOException, InterruptedException {
1424 while (true) {
1425
1426
1427
1428
1429 int count;
1430 if (this.dataLengthBuffer.remaining() > 0) {
1431 count = channelRead(channel, this.dataLengthBuffer);
1432 if (count < 0 || this.dataLengthBuffer.remaining() > 0) {
1433 return count;
1434 }
1435 }
1436
1437 if (!connectionPreambleRead) {
1438
1439 this.dataLengthBuffer.flip();
1440 if (!HConstants.RPC_HEADER.equals(dataLengthBuffer)) {
1441 return doBadPreambleHandling("Expected HEADER=" +
1442 Bytes.toStringBinary(HConstants.RPC_HEADER.array()) +
1443 " but received HEADER=" + Bytes.toStringBinary(dataLengthBuffer.array()));
1444 }
1445
1446 ByteBuffer versionAndAuthBytes = ByteBuffer.allocate(2);
1447 count = channelRead(channel, versionAndAuthBytes);
1448 if (count < 0 || versionAndAuthBytes.remaining() > 0) {
1449 return count;
1450 }
1451 int version = versionAndAuthBytes.get(0);
1452 byte authbyte = versionAndAuthBytes.get(1);
1453 this.authMethod = AuthMethod.valueOf(authbyte);
1454 if (version != CURRENT_VERSION || authMethod == null) {
1455 return doBadPreambleHandling("serverVersion=" + CURRENT_VERSION +
1456 ", clientVersion=" + version + ", authMethod=" + authbyte +
1457 ", authSupported=" + (authMethod != null));
1458 }
1459 if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
1460 AccessControlException ae = new AccessControlException("Authentication is required");
1461 setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
1462 responder.doRespond(authFailedCall);
1463 throw ae;
1464 }
1465 if (!isSecurityEnabled && authMethod != AuthMethod.SIMPLE) {
1466 doRawSaslReply(SaslStatus.SUCCESS, new IntWritable(
1467 SaslUtil.SWITCH_TO_SIMPLE_AUTH), null, null);
1468 authMethod = AuthMethod.SIMPLE;
1469
1470
1471
1472 skipInitialSaslHandshake = true;
1473 }
1474 if (authMethod != AuthMethod.SIMPLE) {
1475 useSasl = true;
1476 }
1477 connectionPreambleRead = true;
1478
1479 dataLengthBuffer.clear();
1480 continue;
1481 }
1482
1483
1484 if (data == null) {
1485 dataLengthBuffer.flip();
1486 int dataLength = dataLengthBuffer.getInt();
1487 if (dataLength == HBaseClient.PING_CALL_ID) {
1488 if (!useWrap) {
1489 dataLengthBuffer.clear();
1490 return 0;
1491 }
1492 }
1493 if (dataLength < 0) {
1494 throw new IllegalArgumentException("Unexpected data length "
1495 + dataLength + "!! from " + getHostAddress());
1496 }
1497 data = ByteBuffer.allocate(dataLength);
1498 incRpcCount();
1499 }
1500 count = channelRead(channel, data);
1501 if (data.remaining() == 0) {
1502 dataLengthBuffer.clear();
1503 data.flip();
1504 if (skipInitialSaslHandshake) {
1505 data = null;
1506 skipInitialSaslHandshake = false;
1507 continue;
1508 }
1509 boolean headerRead = connectionHeaderRead;
1510 if (useSasl) {
1511 saslReadAndProcess(data.array());
1512 } else {
1513 processOneRpc(data.array());
1514 }
1515 this.data = null;
1516 if (!headerRead) {
1517 continue;
1518 }
1519 } else {
1520
1521 if (LOG.isTraceEnabled()) LOG.trace("Continue to read rest of data " + data.remaining());
1522 continue;
1523 }
1524 return count;
1525 }
1526 }
1527
1528 private int doBadPreambleHandling(final String errMsg) throws IOException {
1529 String msg = errMsg + "; cannot communicate with client at " + hostAddress + ":" + port;
1530 LOG.warn(msg);
1531 Call fakeCall = new Call(-1, null, null, null, this, responder, -1, null);
1532 setupResponse(null, fakeCall, new FatalConnectionException(msg), msg);
1533 responder.doRespond(fakeCall);
1534
1535 return -1;
1536 }
1537
1538
1539 private void processConnectionHeader(byte[] buf) throws IOException {
1540 this.connectionHeader = ConnectionHeader.parseFrom(buf);
1541 try {
1542 String protocolClassName = connectionHeader.getProtocol();
1543 if (protocolClassName != null) {
1544 protocol = getProtocolClass(connectionHeader.getProtocol(), conf);
1545 }
1546 } catch (ClassNotFoundException cnfe) {
1547 throw new IOException("Unknown protocol: " + connectionHeader.getProtocol());
1548 }
1549 setupCellBlockCodecs(this.connectionHeader);
1550
1551 UserGroupInformation protocolUser = createUser(connectionHeader);
1552 if (!useSasl) {
1553 user = protocolUser;
1554 if (user != null) {
1555 user.setAuthenticationMethod(AuthMethod.SIMPLE.authenticationMethod);
1556 }
1557 } else {
1558
1559 user.setAuthenticationMethod(authMethod.authenticationMethod);
1560
1561
1562
1563 if ((protocolUser != null)
1564 && (!protocolUser.getUserName().equals(user.getUserName()))) {
1565 if (authMethod == AuthMethod.DIGEST) {
1566
1567 throw new AccessControlException("Authenticated user (" + user
1568 + ") doesn't match what the client claims to be ("
1569 + protocolUser + ")");
1570 } else {
1571
1572
1573
1574 UserGroupInformation realUser = user;
1575 user = UserGroupInformation.createProxyUser(protocolUser
1576 .getUserName(), realUser);
1577
1578 user.setAuthenticationMethod(AuthenticationMethod.PROXY);
1579 }
1580 }
1581 }
1582 }
1583
1584
1585
1586
1587
1588
1589 private void setupCellBlockCodecs(final ConnectionHeader header)
1590 throws FatalConnectionException {
1591
1592 if (!header.hasCellBlockCodecClass()) throw new FatalConnectionException("No codec");
1593 String className = header.getCellBlockCodecClass();
1594 try {
1595 this.codec = (Codec)Class.forName(className).newInstance();
1596 } catch (Exception e) {
1597 throw new FatalConnectionException("Unsupported codec " + className, e);
1598 }
1599 if (!header.hasCellBlockCompressorClass()) return;
1600 className = header.getCellBlockCompressorClass();
1601 try {
1602 this.compressionCodec = (CompressionCodec)Class.forName(className).newInstance();
1603 } catch (Exception e) {
1604 throw new FatalConnectionException("Unsupported codec " + className, e);
1605 }
1606 }
1607
1608 private void processUnwrappedData(byte[] inBuf) throws IOException,
1609 InterruptedException {
1610 ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(
1611 inBuf));
1612
1613 while (true) {
1614 int count = -1;
1615 if (unwrappedDataLengthBuffer.remaining() > 0) {
1616 count = channelRead(ch, unwrappedDataLengthBuffer);
1617 if (count <= 0 || unwrappedDataLengthBuffer.remaining() > 0)
1618 return;
1619 }
1620
1621 if (unwrappedData == null) {
1622 unwrappedDataLengthBuffer.flip();
1623 int unwrappedDataLength = unwrappedDataLengthBuffer.getInt();
1624
1625 if (unwrappedDataLength == HBaseClient.PING_CALL_ID) {
1626 if (LOG.isDebugEnabled())
1627 LOG.debug("Received ping message");
1628 unwrappedDataLengthBuffer.clear();
1629 continue;
1630 }
1631 unwrappedData = ByteBuffer.allocate(unwrappedDataLength);
1632 }
1633
1634 count = channelRead(ch, unwrappedData);
1635 if (count <= 0 || unwrappedData.remaining() > 0)
1636 return;
1637
1638 if (unwrappedData.remaining() == 0) {
1639 unwrappedDataLengthBuffer.clear();
1640 unwrappedData.flip();
1641 processOneRpc(unwrappedData.array());
1642 unwrappedData = null;
1643 }
1644 }
1645 }
1646
1647 private void processOneRpc(byte[] buf) throws IOException,
1648 InterruptedException {
1649 if (connectionHeaderRead) {
1650 processRequest(buf);
1651 } else {
1652 processConnectionHeader(buf);
1653 this.connectionHeaderRead = true;
1654 if (!authorizeConnection()) {
1655 throw new AccessControlException("Connection from " + this
1656 + " for protocol " + connectionHeader.getProtocol()
1657 + " is unauthorized for user " + user);
1658 }
1659 }
1660 }
1661
1662
1663
1664
1665
1666
1667
1668 protected void processRequest(byte[] buf) throws IOException, InterruptedException {
1669 long totalRequestSize = buf.length;
1670 int offset = 0;
1671
1672
1673 CodedInputStream cis = CodedInputStream.newInstance(buf, offset, buf.length);
1674 int headerSize = cis.readRawVarint32();
1675 offset = cis.getTotalBytesRead();
1676 RequestHeader header =
1677 RequestHeader.newBuilder().mergeFrom(buf, offset, headerSize).build();
1678 offset += headerSize;
1679 int id = header.getCallId();
1680 if (LOG.isDebugEnabled()) {
1681 LOG.debug("RequestHeader " + TextFormat.shortDebugString(header) +
1682 " totalRequestSize: " + totalRequestSize + " bytes");
1683 }
1684
1685
1686 if ((totalRequestSize + callQueueSize.get()) > maxQueueSize) {
1687 final Call callTooBig =
1688 new Call(id, null, null, null, this, responder, totalRequestSize, null);
1689 ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1690 setupResponse(responseBuffer, callTooBig, new CallQueueTooBigException(),
1691 "Call queue is full, is ipc.server.max.callqueue.size too small?");
1692 responder.doRespond(callTooBig);
1693 return;
1694 }
1695 Method method = null;
1696 Message param = null;
1697 CellScanner cellScanner = null;
1698 try {
1699 if (header.hasRequestParam() && header.getRequestParam()) {
1700 method = methodCache.getMethod(this.protocol, header.getMethodName());
1701 Message m = methodCache.getMethodArgType(method);
1702
1703 if (m != null) {
1704 Builder builder = null;
1705 builder = m.newBuilderForType();
1706
1707 cis = CodedInputStream.newInstance(buf, offset, buf.length);
1708 int paramSize = cis.readRawVarint32();
1709 offset += cis.getTotalBytesRead();
1710 if (builder != null) {
1711 builder.mergeFrom(buf, offset, paramSize);
1712 param = builder.build();
1713 }
1714 offset += paramSize;
1715 }
1716 }
1717 if (header.hasCellBlockMeta()) {
1718 cellScanner = ipcUtil.createCellScanner(this.codec, this.compressionCodec,
1719 buf, offset, buf.length);
1720 }
1721 } catch (Throwable t) {
1722 String msg = "Unable to read call parameter from client " + getHostAddress();
1723 LOG.warn(msg, t);
1724 final Call readParamsFailedCall =
1725 new Call(id, null, null, null, this, responder, totalRequestSize, null);
1726 ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1727 setupResponse(responseBuffer, readParamsFailedCall, t,
1728 msg + "; " + t.getMessage());
1729 responder.doRespond(readParamsFailedCall);
1730 return;
1731 }
1732
1733 Call call = null;
1734 if (header.hasTraceInfo()) {
1735 call = new Call(id, method, param, cellScanner, this, responder, totalRequestSize,
1736 new TraceInfo(header.getTraceInfo().getTraceId(), header.getTraceInfo().getParentId()));
1737 } else {
1738 call = new Call(id, method, param, cellScanner, this, responder, totalRequestSize, null);
1739 }
1740 callQueueSize.add(totalRequestSize);
1741 Pair<RequestHeader, Message> headerAndParam = new Pair<RequestHeader, Message>(header, param);
1742 if (priorityCallQueue != null && getQosLevel(headerAndParam) > highPriorityLevel) {
1743 priorityCallQueue.put(call);
1744 } else if (replicationQueue != null &&
1745 getQosLevel(headerAndParam) == HConstants.REPLICATION_QOS) {
1746 replicationQueue.put(call);
1747 } else {
1748 callQueue.put(call);
1749 }
1750 }
1751
1752 private boolean authorizeConnection() throws IOException {
1753 try {
1754
1755
1756
1757
1758 if (user != null && user.getRealUser() != null
1759 && (authMethod != AuthMethod.DIGEST)) {
1760 ProxyUsers.authorize(user, this.getHostAddress(), conf);
1761 }
1762 authorize(user, connectionHeader, getHostInetAddress());
1763 if (LOG.isDebugEnabled()) {
1764 LOG.debug("Authorized " + TextFormat.shortDebugString(connectionHeader));
1765 }
1766 metrics.authorizationSuccess();
1767 } catch (AuthorizationException ae) {
1768 LOG.debug("Connection authorization failed: " + ae.getMessage(), ae);
1769 metrics.authorizationFailure();
1770 setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
1771 responder.doRespond(authFailedCall);
1772 return false;
1773 }
1774 return true;
1775 }
1776
1777 protected synchronized void close() {
1778 disposeSasl();
1779 data = null;
1780 this.dataLengthBuffer = null;
1781 if (!channel.isOpen())
1782 return;
1783 try {socket.shutdownOutput();} catch(Exception ignored) {}
1784 if (channel.isOpen()) {
1785 try {channel.close();} catch(Exception ignored) {}
1786 }
1787 try {socket.close();} catch(Exception ignored) {}
1788 }
1789
1790 private UserGroupInformation createUser(ConnectionHeader head) {
1791 UserGroupInformation ugi = null;
1792
1793 if (!head.hasUserInfo()) {
1794 return null;
1795 }
1796 UserInformation userInfoProto = head.getUserInfo();
1797 String effectiveUser = null;
1798 if (userInfoProto.hasEffectiveUser()) {
1799 effectiveUser = userInfoProto.getEffectiveUser();
1800 }
1801 String realUser = null;
1802 if (userInfoProto.hasRealUser()) {
1803 realUser = userInfoProto.getRealUser();
1804 }
1805 if (effectiveUser != null) {
1806 if (realUser != null) {
1807 UserGroupInformation realUserUgi =
1808 UserGroupInformation.createRemoteUser(realUser);
1809 ugi = UserGroupInformation.createProxyUser(effectiveUser, realUserUgi);
1810 } else {
1811 ugi = UserGroupInformation.createRemoteUser(effectiveUser);
1812 }
1813 }
1814 return ugi;
1815 }
1816 }
1817
1818
1819 private class Handler extends Thread {
1820 private final BlockingQueue<Call> myCallQueue;
1821 private MonitoredRPCHandler status;
1822
1823 public Handler(final BlockingQueue<Call> cq, int instanceNumber) {
1824 this.myCallQueue = cq;
1825 this.setDaemon(true);
1826
1827 String threadName = "IPC Server handler " + instanceNumber + " on " + port;
1828 if (cq == priorityCallQueue) {
1829
1830 threadName = "PRI " + threadName;
1831 } else if (cq == replicationQueue) {
1832 threadName = "REPL " + threadName;
1833 }
1834 this.setName(threadName);
1835 this.status = TaskMonitor.get().createRPCStatus(threadName);
1836 }
1837
1838 @Override
1839 public void run() {
1840 LOG.info(getName() + ": starting");
1841 status.setStatus("starting");
1842 SERVER.set(HBaseServer.this);
1843 while (running) {
1844 try {
1845 status.pause("Waiting for a call");
1846 Call call = myCallQueue.take();
1847 status.setStatus("Setting up call");
1848 status.setConnection(call.connection.getHostAddress(), call.connection.getRemotePort());
1849 if (LOG.isDebugEnabled()) {
1850 UserGroupInformation remoteUser = call.connection.user;
1851 LOG.debug(call.toString() + " executing as " +
1852 ((remoteUser == null)? "NULL principal": remoteUser.getUserName()));
1853 }
1854 Throwable errorThrowable = null;
1855 String error = null;
1856 Pair<Message, CellScanner> resultPair = null;
1857 CurCall.set(call);
1858 Span currentRequestSpan = NullSpan.getInstance();
1859 try {
1860 if (!started) {
1861 throw new ServerNotRunningYetException("Server is not running yet");
1862 }
1863 if (call.tinfo != null) {
1864 currentRequestSpan = Trace.startSpan(
1865 "handling " + call.toString(), call.tinfo, Sampler.ALWAYS);
1866 }
1867 RequestContext.set(User.create(call.connection.user), getRemoteIp(),
1868 call.connection.protocol);
1869
1870
1871 resultPair = call(call.connection.protocol, call.method, call.param, call.cellScanner,
1872 call.timestamp, status);
1873 } catch (Throwable e) {
1874 LOG.debug(getName() + ": " + call.toString() + " error: " + e, e);
1875 errorThrowable = e;
1876 error = StringUtils.stringifyException(e);
1877 } finally {
1878 currentRequestSpan.stop();
1879
1880
1881 RequestContext.clear();
1882 }
1883 CurCall.set(null);
1884 callQueueSize.add(call.getSize() * -1);
1885
1886
1887 if (!call.isDelayed() || !call.isReturnValueDelayed()) {
1888 Message param = resultPair != null? resultPair.getFirst(): null;
1889 CellScanner cells = resultPair != null? resultPair.getSecond(): null;
1890 call.setResponse(param, cells, errorThrowable, error);
1891 }
1892 call.sendResponseIfReady();
1893 status.markComplete("Sent response");
1894 } catch (InterruptedException e) {
1895 if (running) {
1896 LOG.info(getName() + ": caught: " + StringUtils.stringifyException(e));
1897 }
1898 } catch (OutOfMemoryError e) {
1899 if (errorHandler != null) {
1900 if (errorHandler.checkOOME(e)) {
1901 LOG.info(getName() + ": exiting on OutOfMemoryError");
1902 return;
1903 }
1904 } else {
1905
1906 throw e;
1907 }
1908 } catch (ClosedChannelException cce) {
1909 LOG.warn(getName() + ": caught a ClosedChannelException, " +
1910 "this means that the server was processing a " +
1911 "request but the client went away. The error message was: " +
1912 cce.getMessage());
1913 } catch (Exception e) {
1914 LOG.warn(getName() + ": caught: " + StringUtils.stringifyException(e));
1915 }
1916 }
1917 LOG.info(getName() + ": exiting");
1918 }
1919 }
1920
1921
1922
1923
1924
1925
1926 protected HBaseServer(String bindAddress, int port,
1927 int handlerCount,
1928 int priorityHandlerCount, Configuration conf, String serverName,
1929 int highPriorityLevel)
1930 throws IOException {
1931 this.bindAddress = bindAddress;
1932 this.conf = conf;
1933 this.port = port;
1934 this.handlerCount = handlerCount;
1935 this.priorityHandlerCount = priorityHandlerCount;
1936 this.socketSendBufferSize = 0;
1937 this.maxQueueLength =
1938 this.conf.getInt("ipc.server.max.callqueue.length",
1939 handlerCount * DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
1940 this.maxQueueSize =
1941 this.conf.getInt("ipc.server.max.callqueue.size",
1942 DEFAULT_MAX_CALLQUEUE_SIZE);
1943 this.readThreads = conf.getInt(
1944 "ipc.server.read.threadpool.size",
1945 10);
1946 this.callQueue = new LinkedBlockingQueue<Call>(maxQueueLength);
1947 if (priorityHandlerCount > 0) {
1948 this.priorityCallQueue = new LinkedBlockingQueue<Call>(maxQueueLength);
1949 } else {
1950 this.priorityCallQueue = null;
1951 }
1952 this.highPriorityLevel = highPriorityLevel;
1953 this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
1954 this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
1955 this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
1956 this.purgeTimeout = conf.getLong("ipc.client.call.purge.timeout",
1957 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
1958 this.numOfReplicationHandlers = conf.getInt("hbase.regionserver.replication.handler.count", 3);
1959 if (numOfReplicationHandlers > 0) {
1960 this.replicationQueue = new LinkedBlockingQueue<Call>(maxQueueSize);
1961 }
1962
1963 listener = new Listener();
1964 this.port = listener.getAddress().getPort();
1965
1966 this.metrics = new MetricsHBaseServer(
1967 serverName, new MetricsHBaseServerWrapperImpl(this));
1968 this.tcpNoDelay = conf.getBoolean("ipc.server.tcpnodelay", true);
1969 this.tcpKeepAlive = conf.getBoolean("ipc.server.tcpkeepalive", true);
1970
1971 this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS,
1972 DEFAULT_WARN_DELAYED_CALLS);
1973 this.delayedCalls = new AtomicInteger(0);
1974 this.ipcUtil = new IPCUtil(conf);
1975
1976
1977
1978 responder = new Responder();
1979 this.authorize =
1980 conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false);
1981 this.isSecurityEnabled = User.isHBaseSecurityEnabled(this.conf);
1982 if (isSecurityEnabled) {
1983 HBaseSaslRpcServer.init(conf);
1984 }
1985 }
1986
1987
1988
1989
1990
1991 protected Connection getConnection(SocketChannel channel, long time) {
1992 return new Connection(channel, time);
1993 }
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005 private void setupResponse(ByteArrayOutputStream response, Call call, Throwable t, String error)
2006 throws IOException {
2007 if (response != null) response.reset();
2008 call.setResponse(null, null, t, error);
2009 }
2010
2011 protected void closeConnection(Connection connection) {
2012 synchronized (connectionList) {
2013 if (connectionList.remove(connection)) {
2014 numConnections--;
2015 }
2016 }
2017 connection.close();
2018 }
2019
2020 Configuration getConf() {
2021 return conf;
2022 }
2023
2024
2025
2026
2027 @Override
2028 public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
2029
2030
2031 @Override
2032 public void start() {
2033 startThreads();
2034 openServer();
2035 }
2036
2037
2038
2039
2040 @Override
2041 public void openServer() {
2042 started = true;
2043 }
2044
2045
2046
2047
2048
2049 @Override
2050 public synchronized void startThreads() {
2051 responder.start();
2052 listener.start();
2053 handlers = startHandlers(callQueue, handlerCount);
2054 priorityHandlers = startHandlers(priorityCallQueue, priorityHandlerCount);
2055 replicationHandlers = startHandlers(replicationQueue, numOfReplicationHandlers);
2056 }
2057
2058 private Handler[] startHandlers(BlockingQueue<Call> queue, int numOfHandlers) {
2059 if (numOfHandlers <= 0) {
2060 return null;
2061 }
2062 Handler[] handlers = new Handler[numOfHandlers];
2063 for (int i = 0; i < numOfHandlers; i++) {
2064 handlers[i] = new Handler(queue, i);
2065 handlers[i].start();
2066 }
2067 return handlers;
2068 }
2069
2070 public SecretManager<? extends TokenIdentifier> getSecretManager() {
2071 return this.secretManager;
2072 }
2073
2074 @SuppressWarnings("unchecked")
2075 public void setSecretManager(SecretManager<? extends TokenIdentifier> secretManager) {
2076 this.secretManager = (SecretManager<TokenIdentifier>) secretManager;
2077 }
2078
2079
2080 @Override
2081 public synchronized void stop() {
2082 LOG.info("Stopping server on " + port);
2083 running = false;
2084 stopHandlers(handlers);
2085 stopHandlers(priorityHandlers);
2086 stopHandlers(replicationHandlers);
2087 listener.interrupt();
2088 listener.doStop();
2089 responder.interrupt();
2090 notifyAll();
2091 }
2092
2093 private void stopHandlers(Handler[] handlers) {
2094 if (handlers != null) {
2095 for (Handler handler : handlers) {
2096 if (handler != null) {
2097 handler.interrupt();
2098 }
2099 }
2100 }
2101 }
2102
2103
2104
2105
2106
2107
2108 @Override
2109 public synchronized void join() throws InterruptedException {
2110 while (running) {
2111 wait();
2112 }
2113 }
2114
2115
2116
2117
2118
2119 @Override
2120 public synchronized InetSocketAddress getListenerAddress() {
2121 return listener.getAddress();
2122 }
2123
2124
2125
2126
2127
2128 @Override
2129 public void setErrorHandler(HBaseRPCErrorHandler handler) {
2130 this.errorHandler = handler;
2131 }
2132
2133
2134
2135
2136 public MetricsHBaseServer getMetrics() {
2137 return metrics;
2138 }
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148 @SuppressWarnings("static-access")
2149 public void authorize(UserGroupInformation user,
2150 ConnectionHeader connection,
2151 InetAddress addr
2152 ) throws AuthorizationException {
2153 if (authorize) {
2154 Class<?> protocol = null;
2155 try {
2156 protocol = getProtocolClass(connection.getProtocol(), getConf());
2157 } catch (ClassNotFoundException cfne) {
2158 throw new AuthorizationException("Unknown protocol: " +
2159 connection.getProtocol());
2160 }
2161 authManager.authorize(user != null ? user : null,
2162 protocol, getConf(), addr);
2163 }
2164 }
2165
2166
2167
2168
2169
2170
2171 private static int NIO_BUFFER_LIMIT = 64 * 1024;
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187 protected int channelWrite(WritableByteChannel channel,
2188 ByteBuffer buffer) throws IOException {
2189
2190 int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
2191 channel.write(buffer) : channelIO(null, channel, buffer);
2192 if (count > 0) {
2193 metrics.sentBytes(count);
2194 }
2195 return count;
2196 }
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210 protected int channelRead(ReadableByteChannel channel,
2211 ByteBuffer buffer) throws IOException {
2212
2213 int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
2214 channel.read(buffer) : channelIO(channel, null, buffer);
2215 if (count > 0) {
2216 metrics.receivedBytes(count);
2217 }
2218 return count;
2219 }
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234 private static int channelIO(ReadableByteChannel readCh,
2235 WritableByteChannel writeCh,
2236 ByteBuffer buf) throws IOException {
2237
2238 int originalLimit = buf.limit();
2239 int initialRemaining = buf.remaining();
2240 int ret = 0;
2241
2242 while (buf.remaining() > 0) {
2243 try {
2244 int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
2245 buf.limit(buf.position() + ioSize);
2246
2247 ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
2248
2249 if (ret < ioSize) {
2250 break;
2251 }
2252
2253 } finally {
2254 buf.limit(originalLimit);
2255 }
2256 }
2257
2258 int nBytes = initialRemaining - buf.remaining();
2259 return (nBytes > 0) ? nBytes : ret;
2260 }
2261
2262
2263
2264
2265
2266
2267 public static RpcCallContext getCurrentCall() {
2268 return CurCall.get();
2269 }
2270 }