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