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