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