1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.ipc;
22
23 import java.io.ByteArrayInputStream;
24 import java.io.ByteArrayOutputStream;
25 import java.io.DataInputStream;
26 import java.io.DataOutputStream;
27 import java.io.IOException;
28 import java.io.InterruptedIOException;
29 import java.net.BindException;
30 import java.net.InetAddress;
31 import java.net.InetSocketAddress;
32 import java.net.ServerSocket;
33 import java.net.Socket;
34 import java.net.SocketException;
35 import java.net.UnknownHostException;
36 import java.nio.ByteBuffer;
37 import java.nio.channels.CancelledKeyException;
38 import java.nio.channels.ClosedChannelException;
39 import java.nio.channels.ReadableByteChannel;
40 import java.nio.channels.SelectionKey;
41 import java.nio.channels.Selector;
42 import java.nio.channels.ServerSocketChannel;
43 import java.nio.channels.SocketChannel;
44 import java.nio.channels.WritableByteChannel;
45 import java.util.ArrayList;
46 import java.util.Collections;
47 import java.util.Iterator;
48 import java.util.LinkedList;
49 import java.util.List;
50 import java.util.Map;
51 import java.util.Random;
52 import java.util.concurrent.BlockingQueue;
53 import java.util.concurrent.ConcurrentHashMap;
54 import java.util.concurrent.ExecutorService;
55 import java.util.concurrent.Executors;
56 import java.util.concurrent.LinkedBlockingQueue;
57 import java.util.concurrent.atomic.AtomicInteger;
58
59 import org.apache.commons.logging.Log;
60 import org.apache.commons.logging.LogFactory;
61 import org.apache.hadoop.conf.Configuration;
62 import org.apache.hadoop.hbase.HConstants;
63 import org.apache.hadoop.hbase.io.HbaseObjectWritable;
64 import org.apache.hadoop.hbase.io.WritableWithSize;
65 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
66 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
67 import org.apache.hadoop.hbase.security.User;
68 import org.apache.hadoop.hbase.util.ByteBufferOutputStream;
69 import org.apache.hadoop.hbase.util.Bytes;
70 import org.apache.hadoop.hbase.util.SizeBasedThrottler;
71 import org.apache.hadoop.io.Writable;
72 import org.apache.hadoop.io.WritableUtils;
73 import org.apache.hadoop.ipc.RPC.VersionMismatch;
74 import org.apache.hadoop.util.ReflectionUtils;
75 import org.apache.hadoop.util.StringUtils;
76 import org.cliffc.high_scale_lib.Counter;
77
78 import com.google.common.base.Function;
79 import com.google.common.util.concurrent.ThreadFactoryBuilder;
80
81
82
83
84
85
86
87
88
89
90 public abstract class HBaseServer implements RpcServer {
91
92
93
94
95 public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
96 public static final byte CURRENT_VERSION = 3;
97
98
99
100
101 private static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
102
103
104
105
106 private static final int DEFAULT_MAX_CALLQUEUE_SIZE =
107 1024 * 1024 * 1024;
108
109 static final int BUFFER_INITIAL_SIZE = 1024;
110
111 private static final String WARN_DELAYED_CALLS =
112 "hbase.ipc.warn.delayedrpc.number";
113
114 private static final int DEFAULT_WARN_DELAYED_CALLS = 1000;
115
116 private final int warnDelayedCalls;
117
118 private AtomicInteger delayedCalls;
119
120 public static final Log LOG =
121 LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer");
122 protected static final Log TRACELOG =
123 LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer.trace");
124
125 protected static final ThreadLocal<RpcServer> SERVER =
126 new ThreadLocal<RpcServer>();
127 private volatile boolean started = false;
128
129 private static final Map<String, Class<? extends VersionedProtocol>>
130 PROTOCOL_CACHE =
131 new ConcurrentHashMap<String, Class<? extends VersionedProtocol>>();
132
133 static Class<? extends VersionedProtocol> getProtocolClass(
134 String protocolName, Configuration conf)
135 throws ClassNotFoundException {
136 Class<? extends VersionedProtocol> protocol =
137 PROTOCOL_CACHE.get(protocolName);
138
139 if (protocol == null) {
140 protocol = (Class<? extends VersionedProtocol>)
141 conf.getClassByName(protocolName);
142 PROTOCOL_CACHE.put(protocolName, protocol);
143 }
144 return protocol;
145 }
146
147
148
149
150
151
152
153 public static RpcServer get() {
154 return SERVER.get();
155 }
156
157
158
159
160 protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
161
162
163
164
165
166 public static InetAddress getRemoteIp() {
167 Call call = CurCall.get();
168 if (call != null) {
169 return call.connection.socket.getInetAddress();
170 }
171 return null;
172 }
173
174
175
176
177 public static String getRemoteAddress() {
178 Call call = CurCall.get();
179 if (call != null) {
180 return call.connection.getHostAddress();
181 }
182 return null;
183 }
184
185 protected String bindAddress;
186 protected int port;
187 private int handlerCount;
188 private int priorityHandlerCount;
189 private int readThreads;
190 protected Class<? extends Writable> paramClass;
191 protected int maxIdleTime;
192
193
194 protected int thresholdIdleConnections;
195
196
197
198 int maxConnectionsToNuke;
199
200
201
202 protected HBaseRpcMetrics rpcMetrics;
203
204 protected Configuration conf;
205
206 private int maxQueueLength;
207 private int maxQueueSize;
208 protected int socketSendBufferSize;
209 protected final boolean tcpNoDelay;
210 protected final boolean tcpKeepAlive;
211 protected final long purgeTimeout;
212
213
214
215 final SizeBasedThrottler responseQueuesSizeThrottler;
216
217
218 private static final long DEFAULT_RESPONSE_QUEUES_MAX_SIZE = 1024 * 1024 * 1024;
219 private static final String RESPONSE_QUEUES_MAX_SIZE = "ipc.server.response.queue.maxsize";
220
221 volatile protected boolean running = true;
222 protected BlockingQueue<Call> callQueue;
223 protected final Counter callQueueSize = new Counter();
224 protected BlockingQueue<Call> priorityCallQueue;
225 private final Counter activeRpcCount = new Counter();
226
227 protected int highPriorityLevel;
228
229 protected final List<Connection> connectionList =
230 Collections.synchronizedList(new LinkedList<Connection>());
231
232
233 private Listener listener = null;
234 protected Responder responder = null;
235 protected int numConnections = 0;
236 private Handler[] handlers = null;
237 private Handler[] priorityHandlers = null;
238
239 protected BlockingQueue<Call> replicationQueue;
240 private int numOfReplicationHandlers = 0;
241 private Handler[] replicationHandlers = null;
242 protected HBaseRPCErrorHandler errorHandler = null;
243
244
245
246
247
248
249
250
251
252
253
254 public static void bind(ServerSocket socket, InetSocketAddress address,
255 int backlog) throws IOException {
256 try {
257 socket.bind(address, backlog);
258 } catch (BindException e) {
259 BindException bindException =
260 new BindException("Problem binding to " + address + " : " +
261 e.getMessage());
262 bindException.initCause(e);
263 throw bindException;
264 } catch (SocketException e) {
265
266
267 if ("Unresolved address".equals(e.getMessage())) {
268 throw new UnknownHostException("Invalid hostname for server: " +
269 address.getHostName());
270 }
271 throw e;
272 }
273 }
274
275
276 protected class Call implements RpcCallContext {
277 protected int id;
278 protected Writable param;
279 protected Connection connection;
280 protected long timestamp;
281
282 protected ByteBuffer response;
283 protected boolean delayResponse;
284 protected Responder responder;
285 protected boolean delayReturnValue;
286
287 protected long size;
288 protected boolean isError;
289
290 public Call(int id, Writable param, Connection connection,
291 Responder responder, long size) {
292 this.id = id;
293 this.param = param;
294 this.connection = connection;
295 this.timestamp = System.currentTimeMillis();
296 this.response = null;
297 this.delayResponse = false;
298 this.responder = responder;
299 this.isError = false;
300 this.size = size;
301 }
302
303 @Override
304 public String toString() {
305 return param.toString() + " from " + connection.toString();
306 }
307
308 protected synchronized void setResponse(Object value, Status status,
309 String errorClass, String error) {
310
311
312
313 if (this.isError)
314 return;
315 if (errorClass != null) {
316 this.isError = true;
317 }
318 Writable result = null;
319 if (value instanceof Writable) {
320 result = (Writable) value;
321 } else {
322
323
324 if (value != null) {
325 result = new HbaseObjectWritable(value);
326 }
327 }
328
329 int size = BUFFER_INITIAL_SIZE;
330 if (result instanceof WritableWithSize) {
331
332 WritableWithSize ohint = (WritableWithSize) result;
333 long hint = ohint.getWritableSize() + Bytes.SIZEOF_BYTE +
334 (2 * Bytes.SIZEOF_INT);
335 if (hint > Integer.MAX_VALUE) {
336
337 IOException ioe =
338 new IOException("Result buffer size too large: " + hint);
339 errorClass = ioe.getClass().getName();
340 error = StringUtils.stringifyException(ioe);
341 } else {
342 size = (int)hint;
343 }
344 }
345
346 ByteBufferOutputStream buf = new ByteBufferOutputStream(size);
347 DataOutputStream out = new DataOutputStream(buf);
348 try {
349
350 out.writeInt(this.id);
351
352 byte flag = (error != null)?
353 ResponseFlag.getErrorAndLengthSet(): ResponseFlag.getLengthSetOnly();
354 out.writeByte(flag);
355
356
357 out.writeInt(0xdeadbeef);
358 out.writeInt(status.state);
359 } catch (IOException e) {
360 errorClass = e.getClass().getName();
361 error = StringUtils.stringifyException(e);
362 }
363
364 try {
365 if (error == null) {
366 result.write(out);
367 } else {
368 WritableUtils.writeString(out, errorClass);
369 WritableUtils.writeString(out, error);
370 }
371 } catch (IOException e) {
372 LOG.warn("Error sending response to call: ", e);
373 }
374
375
376
377 ByteBuffer bb = buf.getByteBuffer();
378 int bufSiz = bb.remaining();
379
380
381 bb.position(Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE);
382 bb.putInt(bufSiz);
383 bb.position(0);
384 this.response = bb;
385 }
386
387 @Override
388 public synchronized void endDelay(Object result) throws IOException {
389 assert this.delayResponse;
390 assert this.delayReturnValue || result == null;
391 this.delayResponse = false;
392 delayedCalls.decrementAndGet();
393 if (this.delayReturnValue)
394 this.setResponse(result, Status.SUCCESS, null, null);
395 this.responder.doRespond(this);
396 }
397
398 @Override
399 public synchronized void endDelay() throws IOException {
400 this.endDelay(null);
401 }
402
403 @Override
404 public synchronized void startDelay(boolean delayReturnValue) {
405 assert !this.delayResponse;
406 this.delayResponse = true;
407 this.delayReturnValue = delayReturnValue;
408 int numDelayed = delayedCalls.incrementAndGet();
409 if (numDelayed > warnDelayedCalls) {
410 LOG.warn("Too many delayed calls: limit " + warnDelayedCalls +
411 " current " + numDelayed);
412 }
413 }
414
415 @Override
416 public synchronized void endDelayThrowing(Throwable t) throws IOException {
417 this.setResponse(null, Status.ERROR, t.getClass().toString(),
418 StringUtils.stringifyException(t));
419 this.delayResponse = false;
420 this.sendResponseIfReady();
421 }
422
423 @Override
424 public synchronized boolean isDelayed() {
425 return this.delayResponse;
426 }
427
428 @Override
429 public synchronized boolean isReturnValueDelayed() {
430 return this.delayReturnValue;
431 }
432
433 @Override
434 public void throwExceptionIfCallerDisconnected() throws CallerDisconnectedException {
435 if (!connection.channel.isOpen()) {
436 long afterTime = System.currentTimeMillis() - timestamp;
437 throw new CallerDisconnectedException(
438 "Aborting call " + this + " after " + afterTime + " ms, since " +
439 "caller disconnected");
440 }
441 }
442
443 public long getSize() {
444 return this.size;
445 }
446
447
448
449
450
451
452 public synchronized void sendResponseIfReady() throws IOException {
453 if (!this.delayResponse) {
454 this.responder.doRespond(this);
455 }
456 }
457 }
458
459
460 private class Listener extends Thread {
461
462 private ServerSocketChannel acceptChannel = null;
463 private Selector selector = null;
464 private Reader[] readers = null;
465 private int currentReader = 0;
466 private InetSocketAddress address;
467 private Random rand = new Random();
468 private long lastCleanupRunTime = 0;
469
470 private long cleanupInterval = 10000;
471
472 private int backlogLength = conf.getInt("ipc.server.listen.queue.size", 128);
473
474 private ExecutorService readPool;
475
476 public Listener() throws IOException {
477 address = new InetSocketAddress(bindAddress, port);
478
479 acceptChannel = ServerSocketChannel.open();
480 acceptChannel.configureBlocking(false);
481
482
483 bind(acceptChannel.socket(), address, backlogLength);
484 port = acceptChannel.socket().getLocalPort();
485
486 selector= Selector.open();
487
488 readers = new Reader[readThreads];
489 readPool = Executors.newFixedThreadPool(readThreads,
490 new ThreadFactoryBuilder().setNameFormat(
491 "IPC Reader %d on port " + port).setDaemon(true).build());
492 for (int i = 0; i < readThreads; ++i) {
493 Reader reader = new Reader();
494 readers[i] = reader;
495 readPool.execute(reader);
496 }
497
498
499 acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
500 this.setName("IPC Server listener on " + port);
501 this.setDaemon(true);
502 }
503
504
505 private class Reader implements Runnable {
506 private volatile boolean adding = false;
507 private final Selector readSelector;
508
509 Reader() throws IOException {
510 this.readSelector = Selector.open();
511 }
512 public void run() {
513 LOG.info("Starting " + getName());
514 try {
515 doRunLoop();
516 } finally {
517 try {
518 readSelector.close();
519 } catch (IOException ioe) {
520 LOG.error("Error closing read selector in " + getName(), ioe);
521 }
522 }
523 }
524
525 private synchronized void doRunLoop() {
526 while (running) {
527 SelectionKey key = null;
528 try {
529 readSelector.select();
530 while (adding) {
531 this.wait(1000);
532 }
533
534 Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
535 while (iter.hasNext()) {
536 key = iter.next();
537 iter.remove();
538 if (key.isValid()) {
539 if (key.isReadable()) {
540 doRead(key);
541 }
542 }
543 key = null;
544 }
545 } catch (InterruptedException e) {
546 if (running) {
547 LOG.info(getName() + " unexpectedly interrupted: " +
548 StringUtils.stringifyException(e));
549 }
550 } catch (IOException ex) {
551 LOG.error("Error in Reader", ex);
552 }
553 }
554 }
555
556
557
558
559
560
561
562
563 public void startAdd() {
564 adding = true;
565 readSelector.wakeup();
566 }
567
568 public synchronized SelectionKey registerChannel(SocketChannel channel)
569 throws IOException {
570 return channel.register(readSelector, SelectionKey.OP_READ);
571 }
572
573 public synchronized void finishAdd() {
574 adding = false;
575 this.notify();
576 }
577 }
578
579
580
581
582
583
584
585
586 private void cleanupConnections(boolean force) {
587 if (force || numConnections > thresholdIdleConnections) {
588 long currentTime = System.currentTimeMillis();
589 if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
590 return;
591 }
592 int start = 0;
593 int end = numConnections - 1;
594 if (!force) {
595 start = rand.nextInt() % numConnections;
596 end = rand.nextInt() % numConnections;
597 int temp;
598 if (end < start) {
599 temp = start;
600 start = end;
601 end = temp;
602 }
603 }
604 int i = start;
605 int numNuked = 0;
606 while (i <= end) {
607 Connection c;
608 synchronized (connectionList) {
609 try {
610 c = connectionList.get(i);
611 } catch (Exception e) {return;}
612 }
613 if (c.timedOut(currentTime)) {
614 if (LOG.isDebugEnabled())
615 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
616 closeConnection(c);
617 numNuked++;
618 end--;
619
620 c = null;
621 if (!force && numNuked == maxConnectionsToNuke) break;
622 }
623 else i++;
624 }
625 lastCleanupRunTime = System.currentTimeMillis();
626 }
627 }
628
629 @Override
630 public void run() {
631 LOG.info(getName() + ": starting");
632 SERVER.set(HBaseServer.this);
633
634 while (running) {
635 SelectionKey key = null;
636 try {
637 selector.select();
638 Iterator<SelectionKey> iter = selector.selectedKeys().iterator();
639 while (iter.hasNext()) {
640 key = iter.next();
641 iter.remove();
642 try {
643 if (key.isValid()) {
644 if (key.isAcceptable())
645 doAccept(key);
646 }
647 } catch (IOException ignored) {
648 }
649 key = null;
650 }
651 } catch (OutOfMemoryError e) {
652 if (errorHandler != null) {
653 if (errorHandler.checkOOME(e)) {
654 LOG.info(getName() + ": exiting on OOME");
655 closeCurrentConnection(key, e);
656 cleanupConnections(true);
657 return;
658 }
659 } else {
660
661
662
663 LOG.warn("Out of Memory in server select", e);
664 closeCurrentConnection(key, e);
665 cleanupConnections(true);
666 try { Thread.sleep(60000); } catch (Exception ignored) {}
667 }
668 } catch (Exception e) {
669 closeCurrentConnection(key, e);
670 }
671 cleanupConnections(false);
672 }
673 LOG.info("Stopping " + this.getName());
674
675 synchronized (this) {
676 try {
677 acceptChannel.close();
678 selector.close();
679 } catch (IOException ignored) { }
680
681 selector= null;
682 acceptChannel= null;
683
684
685 while (!connectionList.isEmpty()) {
686 closeConnection(connectionList.remove(0));
687 }
688 }
689 }
690
691 private void closeCurrentConnection(SelectionKey key, Throwable e) {
692 if (key != null) {
693 Connection c = (Connection)key.attachment();
694 if (c != null) {
695 if (LOG.isDebugEnabled()) {
696 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
697 (e != null ? " on error " + e.getMessage() : ""));
698 }
699 closeConnection(c);
700 key.attach(null);
701 }
702 }
703 }
704
705 InetSocketAddress getAddress() {
706 return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
707 }
708
709 void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
710 Connection c;
711 ServerSocketChannel server = (ServerSocketChannel) key.channel();
712
713 SocketChannel channel;
714 while ((channel = server.accept()) != null) {
715 channel.configureBlocking(false);
716 channel.socket().setTcpNoDelay(tcpNoDelay);
717 channel.socket().setKeepAlive(tcpKeepAlive);
718
719 Reader reader = getReader();
720 try {
721 reader.startAdd();
722 SelectionKey readKey = reader.registerChannel(channel);
723 c = getConnection(channel, System.currentTimeMillis());
724 readKey.attach(c);
725 synchronized (connectionList) {
726 connectionList.add(numConnections, c);
727 numConnections++;
728 }
729 if (LOG.isDebugEnabled())
730 LOG.debug("Server connection from " + c.toString() +
731 "; # active connections: " + numConnections +
732 "; # queued calls: " + callQueue.size());
733 } finally {
734 reader.finishAdd();
735 }
736 }
737 rpcMetrics.numOpenConnections.set(numConnections);
738 }
739
740 void doRead(SelectionKey key) throws InterruptedException {
741 int count = 0;
742 Connection c = (Connection)key.attachment();
743 if (c == null) {
744 return;
745 }
746 c.setLastContact(System.currentTimeMillis());
747
748 try {
749 count = c.readAndProcess();
750 } catch (InterruptedException ieo) {
751 throw ieo;
752 } catch (Exception e) {
753 LOG.warn(getName() + ": readAndProcess threw exception " + e + ". Count of bytes read: " + count, e);
754 count = -1;
755 }
756 if (count < 0) {
757 if (LOG.isDebugEnabled())
758 LOG.debug(getName() + ": disconnecting client " +
759 c.getHostAddress() + ". Number of active connections: "+
760 numConnections);
761 closeConnection(c);
762
763 }
764 else {
765 c.setLastContact(System.currentTimeMillis());
766 }
767 }
768
769 synchronized void doStop() {
770 if (selector != null) {
771 selector.wakeup();
772 Thread.yield();
773 }
774 if (acceptChannel != null) {
775 try {
776 acceptChannel.socket().close();
777 } catch (IOException e) {
778 LOG.info(getName() + ":Exception in closing listener socket. " + e);
779 }
780 }
781 readPool.shutdownNow();
782 }
783
784
785
786 Reader getReader() {
787 currentReader = (currentReader + 1) % readers.length;
788 return readers[currentReader];
789 }
790 }
791
792
793 protected class Responder extends Thread {
794 private final Selector writeSelector;
795 private int pending;
796
797 Responder() throws IOException {
798 this.setName("IPC Server Responder");
799 this.setDaemon(true);
800 writeSelector = Selector.open();
801 pending = 0;
802 }
803
804 @Override
805 public void run() {
806 LOG.info(getName() + ": starting");
807 SERVER.set(HBaseServer.this);
808 try {
809 doRunLoop();
810 } finally {
811 LOG.info("Stopping " + this.getName());
812 try {
813 writeSelector.close();
814 } catch (IOException ioe) {
815 LOG.error("Couldn't close write selector in " + this.getName(), ioe);
816 }
817 }
818 }
819
820 private void doRunLoop() {
821 long lastPurgeTime = 0;
822
823 while (running) {
824 try {
825 waitPending();
826 writeSelector.select(purgeTimeout);
827 Iterator<SelectionKey> iter = writeSelector.selectedKeys().iterator();
828 while (iter.hasNext()) {
829 SelectionKey key = iter.next();
830 iter.remove();
831 try {
832 if (key.isValid() && key.isWritable()) {
833 doAsyncWrite(key);
834 }
835 } catch (IOException e) {
836 LOG.info(getName() + ": doAsyncWrite threw exception " + e);
837 }
838 }
839 long now = System.currentTimeMillis();
840 if (now < lastPurgeTime + purgeTimeout) {
841 continue;
842 }
843 lastPurgeTime = now;
844
845
846
847
848 LOG.debug("Checking for old call responses.");
849 ArrayList<Call> calls;
850
851
852 synchronized (writeSelector.keys()) {
853 calls = new ArrayList<Call>(writeSelector.keys().size());
854 iter = writeSelector.keys().iterator();
855 while (iter.hasNext()) {
856 SelectionKey key = iter.next();
857 Call call = (Call)key.attachment();
858 if (call != null && key.channel() == call.connection.channel) {
859 calls.add(call);
860 }
861 }
862 }
863
864 for(Call call : calls) {
865 try {
866 doPurge(call, now);
867 } catch (IOException e) {
868 LOG.warn("Error in purging old calls " + e);
869 }
870 }
871 } catch (OutOfMemoryError e) {
872 if (errorHandler != null) {
873 if (errorHandler.checkOOME(e)) {
874 LOG.info(getName() + ": exiting on OOME");
875 return;
876 }
877 } else {
878
879
880
881
882
883 LOG.warn("Out of Memory in server select", e);
884 try { Thread.sleep(60000); } catch (Exception ignored) {}
885 }
886 } catch (Exception e) {
887 LOG.warn("Exception in Responder " +
888 StringUtils.stringifyException(e));
889 }
890 }
891 LOG.info("Stopping " + this.getName());
892 }
893
894 private void doAsyncWrite(SelectionKey key) throws IOException {
895 Call call = (Call)key.attachment();
896 if (call == null) {
897 return;
898 }
899 if (key.channel() != call.connection.channel) {
900 throw new IOException("doAsyncWrite: bad channel");
901 }
902
903 synchronized(call.connection.responseQueue) {
904 if (processResponse(call.connection.responseQueue, false)) {
905 try {
906 key.interestOps(0);
907 } catch (CancelledKeyException e) {
908
909
910
911
912
913 LOG.warn("Exception while changing ops : " + e);
914 }
915 }
916 }
917 }
918
919
920
921
922
923 private void doPurge(Call call, long now) throws IOException {
924 synchronized (call.connection.responseQueue) {
925 Iterator<Call> iter = call.connection.responseQueue.listIterator(0);
926 while (iter.hasNext()) {
927 Call nextCall = iter.next();
928 if (now > nextCall.timestamp + purgeTimeout) {
929 closeConnection(nextCall.connection);
930 break;
931 }
932 }
933 }
934 }
935
936
937
938
939 private boolean processResponse(final LinkedList<Call> responseQueue,
940 boolean inHandler) throws IOException {
941 boolean error = true;
942 boolean done = false;
943 int numElements;
944 Call call = null;
945 try {
946
947 synchronized (responseQueue) {
948
949
950
951 numElements = responseQueue.size();
952 if (numElements == 0) {
953 error = false;
954 return true;
955 }
956
957
958
959 call = responseQueue.peek();
960 SocketChannel channel = call.connection.channel;
961 if (LOG.isDebugEnabled()) {
962 LOG.debug(getName() + ": responding to #" + call.id + " from " +
963 call.connection);
964 }
965
966
967
968 int numBytes = channelWrite(channel, call.response);
969 if (numBytes < 0) {
970
971
972 return true;
973 }
974 if (!call.response.hasRemaining()) {
975 responseQueue.poll();
976 responseQueuesSizeThrottler.decrease(call.response.limit());
977 call.connection.decRpcCount();
978
979 if (numElements == 1) {
980 done = true;
981 } else {
982 done = false;
983 }
984 if (LOG.isDebugEnabled()) {
985 LOG.debug(getName() + ": responding to #" + call.id + " from " +
986 call.connection + " Wrote " + numBytes + " bytes.");
987 }
988 } else {
989 if (inHandler) {
990
991 call.timestamp = System.currentTimeMillis();
992 if (enqueueInSelector(call))
993 done = true;
994 }
995 if (LOG.isDebugEnabled()) {
996 LOG.debug(getName() + ": responding to #" + call.id + " from " +
997 call.connection + " Wrote partial " + numBytes +
998 " bytes.");
999 }
1000 }
1001 error = false;
1002 }
1003 } finally {
1004 if (error && call != null) {
1005 LOG.warn(getName()+", call " + call + ": output error");
1006 done = true;
1007 closeConnection(call.connection);
1008 }
1009 }
1010 return done;
1011 }
1012
1013
1014
1015
1016 private boolean enqueueInSelector(Call call) throws IOException {
1017 boolean done = false;
1018 incPending();
1019 try {
1020
1021
1022 SocketChannel channel = call.connection.channel;
1023 writeSelector.wakeup();
1024 channel.register(writeSelector, SelectionKey.OP_WRITE, call);
1025 } catch (ClosedChannelException e) {
1026
1027 done = true;
1028 } finally {
1029 decPending();
1030 }
1031 return done;
1032 }
1033
1034
1035
1036
1037 void doRespond(Call call) throws IOException {
1038
1039 call.timestamp = System.currentTimeMillis();
1040
1041 boolean doRegister = false;
1042 boolean closed;
1043 try {
1044 responseQueuesSizeThrottler.increase(call.response.remaining());
1045 } catch (InterruptedException ie) {
1046 throw new InterruptedIOException(ie.getMessage());
1047 }
1048 synchronized (call.connection.responseQueue) {
1049 closed = call.connection.closed;
1050 if (!closed) {
1051 call.connection.responseQueue.addLast(call);
1052
1053 if (call.connection.responseQueue.size() == 1) {
1054 doRegister = !processResponse(call.connection.responseQueue, false);
1055 }
1056 }
1057 }
1058 if (doRegister) {
1059 enqueueInSelector(call);
1060 }
1061 if (closed) {
1062
1063
1064
1065 responseQueuesSizeThrottler.decrease(call.response.remaining());
1066 }
1067 }
1068
1069 private synchronized void incPending() {
1070 pending++;
1071 }
1072
1073 private synchronized void decPending() {
1074 pending--;
1075 notify();
1076 }
1077
1078 private synchronized void waitPending() throws InterruptedException {
1079 while (pending > 0) {
1080 wait();
1081 }
1082 }
1083 }
1084
1085
1086 protected class Connection {
1087 private boolean versionRead = false;
1088
1089 private boolean headerRead = false;
1090
1091
1092 protected volatile boolean closed = false;
1093 protected SocketChannel channel;
1094 private ByteBuffer data;
1095 private ByteBuffer dataLengthBuffer;
1096 protected final LinkedList<Call> responseQueue;
1097 private volatile int rpcCount = 0;
1098 private long lastContact;
1099 private int dataLength;
1100 protected Socket socket;
1101
1102
1103 protected String hostAddress;
1104 protected int remotePort;
1105 ConnectionHeader header = new ConnectionHeader();
1106 Class<? extends VersionedProtocol> protocol;
1107 protected User ticket = null;
1108
1109 public Connection(SocketChannel channel, long lastContact) {
1110 this.channel = channel;
1111 this.lastContact = lastContact;
1112 this.data = null;
1113 this.dataLengthBuffer = ByteBuffer.allocate(4);
1114 this.socket = channel.socket();
1115 InetAddress addr = socket.getInetAddress();
1116 if (addr == null) {
1117 this.hostAddress = "*Unknown*";
1118 } else {
1119 this.hostAddress = addr.getHostAddress();
1120 }
1121 this.remotePort = socket.getPort();
1122 this.responseQueue = new LinkedList<Call>();
1123 if (socketSendBufferSize != 0) {
1124 try {
1125 socket.setSendBufferSize(socketSendBufferSize);
1126 } catch (IOException e) {
1127 LOG.warn("Connection: unable to set socket send buffer size to " +
1128 socketSendBufferSize);
1129 }
1130 }
1131 }
1132
1133 @Override
1134 public String toString() {
1135 return getHostAddress() + ":" + remotePort;
1136 }
1137
1138 public String getHostAddress() {
1139 return hostAddress;
1140 }
1141
1142 public int getRemotePort() {
1143 return remotePort;
1144 }
1145
1146 public void setLastContact(long lastContact) {
1147 this.lastContact = lastContact;
1148 }
1149
1150 public long getLastContact() {
1151 return lastContact;
1152 }
1153
1154
1155 private boolean isIdle() {
1156 return rpcCount == 0;
1157 }
1158
1159
1160 protected void decRpcCount() {
1161 rpcCount--;
1162 }
1163
1164
1165 protected void incRpcCount() {
1166 rpcCount++;
1167 }
1168
1169 protected boolean timedOut(long currentTime) {
1170 return isIdle() && currentTime - lastContact > maxIdleTime;
1171 }
1172
1173 public int readAndProcess() throws IOException, InterruptedException {
1174 while (true) {
1175
1176
1177
1178 int count;
1179 if (dataLengthBuffer.remaining() > 0) {
1180 count = channelRead(channel, dataLengthBuffer);
1181 if (count < 0 || dataLengthBuffer.remaining() > 0)
1182 return count;
1183 }
1184
1185 if (!versionRead) {
1186
1187 ByteBuffer versionBuffer = ByteBuffer.allocate(1);
1188 count = channelRead(channel, versionBuffer);
1189 if (count <= 0) {
1190 return count;
1191 }
1192 int version = versionBuffer.get(0);
1193
1194 dataLengthBuffer.flip();
1195 if (!HEADER.equals(dataLengthBuffer) || version != CURRENT_VERSION) {
1196
1197 LOG.warn("Incorrect header or version mismatch from " +
1198 hostAddress + ":" + remotePort +
1199 " got version " + version +
1200 " expected version " + CURRENT_VERSION);
1201 setupBadVersionResponse(version);
1202 return -1;
1203 }
1204 dataLengthBuffer.clear();
1205 versionRead = true;
1206 continue;
1207 }
1208
1209 if (data == null) {
1210 dataLengthBuffer.flip();
1211 dataLength = dataLengthBuffer.getInt();
1212
1213 if (dataLength == HBaseClient.PING_CALL_ID) {
1214 dataLengthBuffer.clear();
1215 return 0;
1216 }
1217 data = ByteBuffer.allocate(dataLength);
1218 incRpcCount();
1219 }
1220
1221 count = channelRead(channel, data);
1222
1223 if (data.remaining() == 0) {
1224 dataLengthBuffer.clear();
1225 data.flip();
1226 if (headerRead) {
1227 processData(data.array());
1228 data = null;
1229 return count;
1230 }
1231 processHeader();
1232 headerRead = true;
1233 data = null;
1234 continue;
1235 }
1236 return count;
1237 }
1238 }
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248 private void setupBadVersionResponse(int clientVersion) throws IOException {
1249 String errMsg = "Server IPC version " + CURRENT_VERSION +
1250 " cannot communicate with client version " + clientVersion;
1251 ByteArrayOutputStream buffer = new ByteArrayOutputStream();
1252
1253 if (clientVersion >= 3) {
1254
1255
1256
1257
1258
1259 Call fakeCall = new Call(0, null, this, responder, 0);
1260
1261
1262 setupResponse(buffer, fakeCall, Status.FATAL,
1263 null, VersionMismatch.class.getName(), errMsg);
1264
1265 responder.doRespond(fakeCall);
1266 }
1267 }
1268
1269
1270 private void processHeader() throws IOException {
1271 DataInputStream in =
1272 new DataInputStream(new ByteArrayInputStream(data.array()));
1273 header.readFields(in);
1274 try {
1275 String protocolClassName = header.getProtocol();
1276 if (protocolClassName == null) {
1277 protocolClassName = "org.apache.hadoop.hbase.ipc.HRegionInterface";
1278 }
1279 protocol = getProtocolClass(protocolClassName, conf);
1280 } catch (ClassNotFoundException cnfe) {
1281 throw new IOException("Unknown protocol: " + header.getProtocol());
1282 }
1283
1284 ticket = header.getUser();
1285 }
1286
1287 protected void processData(byte[] buf) throws IOException, InterruptedException {
1288 DataInputStream dis =
1289 new DataInputStream(new ByteArrayInputStream(buf));
1290 int id = dis.readInt();
1291 long callSize = buf.length;
1292
1293 if (LOG.isDebugEnabled()) {
1294 LOG.debug(" got call #" + id + ", " + callSize + " bytes");
1295 }
1296
1297
1298 if ((callSize + callQueueSize.get()) > maxQueueSize) {
1299 final Call callTooBig =
1300 new Call(id, null, this, responder, callSize);
1301 ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1302 setupResponse(responseBuffer, callTooBig, Status.FATAL, null,
1303 IOException.class.getName(),
1304 "Call queue is full, is ipc.server.max.callqueue.size too small?");
1305 responder.doRespond(callTooBig);
1306 return;
1307 }
1308
1309 Writable param;
1310 try {
1311 param = ReflectionUtils.newInstance(paramClass, conf);
1312 param.readFields(dis);
1313 } catch (Throwable t) {
1314 LOG.warn("Unable to read call parameters for client " +
1315 getHostAddress(), t);
1316 final Call readParamsFailedCall =
1317 new Call(id, null, this, responder, callSize);
1318 ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1319
1320 setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
1321 t.getClass().getName(),
1322 "IPC server unable to read call parameters: " + t.getMessage());
1323 responder.doRespond(readParamsFailedCall);
1324 return;
1325 }
1326 Call call = new Call(id, param, this, responder, callSize);
1327 callQueueSize.add(callSize);
1328
1329 if (priorityCallQueue != null && getQosLevel(param) > highPriorityLevel) {
1330 priorityCallQueue.put(call);
1331 updateCallQueueLenMetrics(priorityCallQueue);
1332 } else if (replicationQueue != null && getQosLevel(param) == HConstants.REPLICATION_QOS) {
1333 replicationQueue.put(call);
1334 updateCallQueueLenMetrics(replicationQueue);
1335 } else {
1336 callQueue.put(call);
1337 updateCallQueueLenMetrics(callQueue);
1338 }
1339 }
1340
1341 protected synchronized void close() {
1342 closed = true;
1343 data = null;
1344 dataLengthBuffer = null;
1345 if (!channel.isOpen())
1346 return;
1347 try {socket.shutdownOutput();} catch(Exception ignored) {}
1348 if (channel.isOpen()) {
1349 try {channel.close();} catch(Exception ignored) {}
1350 }
1351 try {socket.close();} catch(Exception ignored) {}
1352 }
1353 }
1354
1355
1356
1357
1358
1359 protected void updateCallQueueLenMetrics(BlockingQueue<Call> queue) {
1360 if (queue == callQueue) {
1361 rpcMetrics.callQueueLen.set(callQueue.size());
1362 } else if (queue == priorityCallQueue) {
1363 rpcMetrics.priorityCallQueueLen.set(priorityCallQueue.size());
1364 } else if (queue == replicationQueue) {
1365 rpcMetrics.replicationCallQueueLen.set(replicationQueue.size());
1366 } else {
1367 LOG.warn("Unknown call queue");
1368 }
1369 }
1370
1371
1372 private class Handler extends Thread {
1373 private final BlockingQueue<Call> myCallQueue;
1374 private MonitoredRPCHandler status;
1375
1376 public Handler(final BlockingQueue<Call> cq, int instanceNumber) {
1377 this.myCallQueue = cq;
1378 this.setDaemon(true);
1379
1380 String threadName = "IPC Server handler " + instanceNumber + " on " + port;
1381 if (cq == priorityCallQueue) {
1382
1383 threadName = "PRI " + threadName;
1384 } else if (cq == replicationQueue) {
1385 threadName = "REPL " + threadName;
1386 }
1387 this.setName(threadName);
1388 this.status = TaskMonitor.get().createRPCStatus(threadName);
1389 }
1390
1391 @Override
1392 public void run() {
1393 LOG.info(getName() + ": starting");
1394 status.setStatus("starting");
1395 SERVER.set(HBaseServer.this);
1396 while (running) {
1397 try {
1398 status.pause("Waiting for a call");
1399 Call call = myCallQueue.take();
1400 updateCallQueueLenMetrics(myCallQueue);
1401 if (!call.connection.channel.isOpen()) {
1402 if (LOG.isDebugEnabled()) {
1403 LOG.debug(Thread.currentThread().getName() + ": skipped " + call);
1404 }
1405 continue;
1406 }
1407 status.setStatus("Setting up call");
1408 status.setConnection(call.connection.getHostAddress(),
1409 call.connection.getRemotePort());
1410
1411 if (LOG.isDebugEnabled())
1412 LOG.debug(getName() + ": has #" + call.id + " from " +
1413 call.connection);
1414
1415 String errorClass = null;
1416 String error = null;
1417 Writable value = null;
1418
1419 CurCall.set(call);
1420 try {
1421 activeRpcCount.increment();
1422 if (!started)
1423 throw new ServerNotRunningYetException("Server is not running yet");
1424
1425 if (LOG.isDebugEnabled()) {
1426 User remoteUser = call.connection.ticket;
1427 LOG.debug(getName() + ": call #" + call.id + " executing as "
1428 + (remoteUser == null ? "NULL principal" : remoteUser.getName()));
1429 }
1430
1431 RequestContext.set(call.connection.ticket, getRemoteIp(),
1432 call.connection.protocol);
1433
1434 value = call(call.connection.protocol, call.param, call.timestamp,
1435 status);
1436 } catch (Throwable e) {
1437 LOG.debug(getName()+", call "+call+": error: " + e, e);
1438 errorClass = e.getClass().getName();
1439 error = StringUtils.stringifyException(e);
1440 } finally {
1441
1442
1443 RequestContext.clear();
1444 activeRpcCount.decrement();
1445 rpcMetrics.activeRpcCount.set((int) activeRpcCount.get());
1446 }
1447 CurCall.set(null);
1448 callQueueSize.add(call.getSize() * -1);
1449
1450
1451 if (!call.isDelayed() || !call.isReturnValueDelayed()) {
1452 call.setResponse(value,
1453 errorClass == null? Status.SUCCESS: Status.ERROR,
1454 errorClass, error);
1455 }
1456 call.sendResponseIfReady();
1457 status.markComplete("Sent response");
1458 } catch (InterruptedException e) {
1459 if (running) {
1460 LOG.info(getName() + " caught: " +
1461 StringUtils.stringifyException(e));
1462 }
1463 } catch (OutOfMemoryError e) {
1464 if (errorHandler != null) {
1465 if (errorHandler.checkOOME(e)) {
1466 LOG.info(getName() + ": exiting on OOME");
1467 return;
1468 }
1469 } else {
1470
1471 throw e;
1472 }
1473 } catch (ClosedChannelException cce) {
1474 LOG.warn(getName() + " caught a ClosedChannelException, " +
1475 "this means that the server was processing a " +
1476 "request but the client went away. The error message was: " +
1477 cce.getMessage());
1478 } catch (Exception e) {
1479 LOG.warn(getName() + " caught: " +
1480 StringUtils.stringifyException(e));
1481 }
1482 }
1483 LOG.info(getName() + ": exiting");
1484 }
1485
1486 }
1487
1488
1489 private Function<Writable,Integer> qosFunction = null;
1490
1491
1492
1493
1494
1495
1496
1497 @Override
1498 public void setQosFunction(Function<Writable, Integer> newFunc) {
1499 qosFunction = newFunc;
1500 }
1501
1502 protected int getQosLevel(Writable param) {
1503 if (qosFunction == null) {
1504 return 0;
1505 }
1506
1507 Integer res = qosFunction.apply(param);
1508 if (res == null) {
1509 return 0;
1510 }
1511 return res;
1512 }
1513
1514
1515
1516
1517
1518
1519 protected HBaseServer(String bindAddress, int port,
1520 Class<? extends Writable> paramClass, int handlerCount,
1521 int priorityHandlerCount, Configuration conf, String serverName,
1522 int highPriorityLevel)
1523 throws IOException {
1524 this.bindAddress = bindAddress;
1525 this.conf = conf;
1526 this.port = port;
1527 this.paramClass = paramClass;
1528 this.handlerCount = handlerCount;
1529 this.priorityHandlerCount = priorityHandlerCount;
1530 this.socketSendBufferSize = 0;
1531
1532
1533 String oldMaxQueueSize = this.conf.get("ipc.server.max.queue.size");
1534 if (oldMaxQueueSize == null) {
1535 this.maxQueueLength =
1536 this.conf.getInt("ipc.server.max.callqueue.length",
1537 handlerCount * DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
1538 } else {
1539 LOG.warn("ipc.server.max.queue.size was renamed " +
1540 "ipc.server.max.callqueue.length, " +
1541 "please update your configuration");
1542 this.maxQueueLength = Integer.getInteger(oldMaxQueueSize);
1543 }
1544
1545 this.maxQueueSize =
1546 this.conf.getInt("ipc.server.max.callqueue.size",
1547 DEFAULT_MAX_CALLQUEUE_SIZE);
1548 this.readThreads = conf.getInt(
1549 "ipc.server.read.threadpool.size",
1550 10);
1551 this.callQueue = new LinkedBlockingQueue<Call>(maxQueueLength);
1552 if (priorityHandlerCount > 0) {
1553 this.priorityCallQueue = new LinkedBlockingQueue<Call>(maxQueueLength);
1554 } else {
1555 this.priorityCallQueue = null;
1556 }
1557 this.highPriorityLevel = highPriorityLevel;
1558 this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
1559 this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
1560 this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
1561 this.purgeTimeout = conf.getLong("ipc.client.call.purge.timeout",
1562 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
1563 this.numOfReplicationHandlers =
1564 conf.getInt("hbase.regionserver.replication.handler.count", 3);
1565 if (numOfReplicationHandlers > 0) {
1566 this.replicationQueue = new LinkedBlockingQueue<Call>(maxQueueLength);
1567 }
1568
1569 listener = new Listener();
1570 this.port = listener.getAddress().getPort();
1571 this.rpcMetrics = new HBaseRpcMetrics(
1572 serverName, Integer.toString(this.port));
1573 this.tcpNoDelay = conf.getBoolean("ipc.server.tcpnodelay", false);
1574 this.tcpKeepAlive = conf.getBoolean("ipc.server.tcpkeepalive", true);
1575
1576 this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS,
1577 DEFAULT_WARN_DELAYED_CALLS);
1578 this.delayedCalls = new AtomicInteger(0);
1579
1580
1581 this.responseQueuesSizeThrottler = new SizeBasedThrottler(
1582 conf.getLong(RESPONSE_QUEUES_MAX_SIZE, DEFAULT_RESPONSE_QUEUES_MAX_SIZE));
1583
1584
1585 responder = new Responder();
1586 }
1587
1588
1589
1590
1591
1592 protected Connection getConnection(SocketChannel channel, long time) {
1593 return new Connection(channel, time);
1594 }
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607 private void setupResponse(ByteArrayOutputStream response,
1608 Call call, Status status,
1609 Writable rv, String errorClass, String error)
1610 throws IOException {
1611 response.reset();
1612 DataOutputStream out = new DataOutputStream(response);
1613
1614 if (status == Status.SUCCESS) {
1615 try {
1616 rv.write(out);
1617 call.setResponse(rv, status, null, null);
1618 } catch (Throwable t) {
1619 LOG.warn("Error serializing call response for call " + call, t);
1620
1621
1622
1623 call.setResponse(null, status.ERROR, t.getClass().getName(),
1624 StringUtils.stringifyException(t));
1625 }
1626 } else {
1627 call.setResponse(rv, status, errorClass, error);
1628 }
1629 }
1630
1631 protected void closeConnection(Connection connection) {
1632 synchronized (connectionList) {
1633 if (connectionList.remove(connection)) {
1634 numConnections--;
1635 }
1636 }
1637 connection.close();
1638 long bytes = 0;
1639 synchronized (connection.responseQueue) {
1640 for (Call c : connection.responseQueue) {
1641 bytes += c.response.limit();
1642 }
1643 connection.responseQueue.clear();
1644 }
1645 responseQueuesSizeThrottler.decrease(bytes);
1646 rpcMetrics.numOpenConnections.set(numConnections);
1647 }
1648
1649
1650
1651
1652 @Override
1653 public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
1654
1655
1656 @Override
1657 public void start() {
1658 startThreads();
1659 openServer();
1660 }
1661
1662
1663
1664
1665 @Override
1666 public void openServer() {
1667 started = true;
1668 }
1669
1670
1671
1672
1673
1674 @Override
1675 public synchronized void startThreads() {
1676 responder.start();
1677 listener.start();
1678 handlers = startHandlers(callQueue, handlerCount);
1679 priorityHandlers = startHandlers(priorityCallQueue, priorityHandlerCount);
1680 replicationHandlers = startHandlers(replicationQueue, numOfReplicationHandlers);
1681 }
1682
1683 private Handler[] startHandlers(BlockingQueue<Call> queue, int numOfHandlers) {
1684 if (numOfHandlers <= 0) {
1685 return null;
1686 }
1687 Handler[] handlers = new Handler[numOfHandlers];
1688 for (int i = 0; i < numOfHandlers; i++) {
1689 handlers[i] = new Handler(queue, i);
1690 handlers[i].start();
1691 }
1692 return handlers;
1693 }
1694
1695
1696 @Override
1697 public synchronized void stop() {
1698 LOG.info("Stopping server on " + port);
1699 running = false;
1700 stopHandlers(handlers);
1701 stopHandlers(priorityHandlers);
1702 stopHandlers(replicationHandlers);
1703 listener.interrupt();
1704 listener.doStop();
1705 responder.interrupt();
1706 notifyAll();
1707 if (this.rpcMetrics != null) {
1708 this.rpcMetrics.shutdown();
1709 }
1710 }
1711
1712 private void stopHandlers(Handler[] handlers) {
1713 if (handlers != null) {
1714 for (Handler handler : handlers) {
1715 if (handler != null) {
1716 handler.interrupt();
1717 }
1718 }
1719 }
1720 }
1721
1722
1723
1724
1725
1726
1727 @Override
1728 public synchronized void join() throws InterruptedException {
1729 while (running) {
1730 wait();
1731 }
1732 }
1733
1734
1735
1736
1737
1738 @Override
1739 public synchronized InetSocketAddress getListenerAddress() {
1740 return listener.getAddress();
1741 }
1742
1743
1744
1745
1746
1747 @Override
1748 public void setErrorHandler(HBaseRPCErrorHandler handler) {
1749 this.errorHandler = handler;
1750 }
1751
1752
1753
1754
1755 public HBaseRpcMetrics getRpcMetrics() {
1756 return rpcMetrics;
1757 }
1758
1759
1760
1761
1762
1763
1764 private static int NIO_BUFFER_LIMIT = 64 * 1024;
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780 protected int channelWrite(WritableByteChannel channel,
1781 ByteBuffer buffer) throws IOException {
1782
1783 int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1784 channel.write(buffer) : channelIO(null, channel, buffer);
1785 if (count > 0) {
1786 rpcMetrics.sentBytes.inc(count);
1787 }
1788 return count;
1789 }
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803 protected int channelRead(ReadableByteChannel channel,
1804 ByteBuffer buffer) throws IOException {
1805
1806 int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1807 channel.read(buffer) : channelIO(channel, null, buffer);
1808 if (count > 0) {
1809 rpcMetrics.receivedBytes.inc(count);
1810 }
1811 return count;
1812 }
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827 private static int channelIO(ReadableByteChannel readCh,
1828 WritableByteChannel writeCh,
1829 ByteBuffer buf) throws IOException {
1830
1831 int originalLimit = buf.limit();
1832 int initialRemaining = buf.remaining();
1833 int ret = 0;
1834
1835 while (buf.remaining() > 0) {
1836 try {
1837 int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
1838 buf.limit(buf.position() + ioSize);
1839
1840 ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
1841
1842 if (ret < ioSize) {
1843 break;
1844 }
1845
1846 } finally {
1847 buf.limit(originalLimit);
1848 }
1849 }
1850
1851 int nBytes = initialRemaining - buf.remaining();
1852 return (nBytes > 0) ? nBytes : ret;
1853 }
1854
1855
1856
1857
1858
1859
1860 public static RpcCallContext getCurrentCall() {
1861 return CurCall.get();
1862 }
1863
1864 public long getResponseQueueSize(){
1865 return responseQueuesSizeThrottler.getCurrentValue();
1866 }
1867 }