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