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