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