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 com.google.common.base.Function;
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.util.Bytes;
28 import org.apache.hadoop.io.ObjectWritable;
29 import org.apache.hadoop.io.Writable;
30 import org.apache.hadoop.io.WritableUtils;
31 import org.apache.hadoop.security.UserGroupInformation;
32 import org.apache.hadoop.util.ReflectionUtils;
33 import org.apache.hadoop.util.StringUtils;
34
35 import java.io.ByteArrayInputStream;
36 import java.io.ByteArrayOutputStream;
37 import java.io.DataInputStream;
38 import java.io.DataOutputStream;
39 import java.io.IOException;
40 import java.net.BindException;
41 import java.net.InetAddress;
42 import java.net.InetSocketAddress;
43 import java.net.ServerSocket;
44 import java.net.Socket;
45 import java.net.SocketException;
46 import java.net.UnknownHostException;
47 import java.nio.ByteBuffer;
48 import java.nio.channels.CancelledKeyException;
49 import java.nio.channels.ClosedChannelException;
50 import java.nio.channels.ReadableByteChannel;
51 import java.nio.channels.SelectionKey;
52 import java.nio.channels.Selector;
53 import java.nio.channels.ServerSocketChannel;
54 import java.nio.channels.SocketChannel;
55 import java.nio.channels.WritableByteChannel;
56 import java.util.ArrayList;
57 import java.util.Collections;
58 import java.util.Iterator;
59 import java.util.LinkedList;
60 import java.util.List;
61 import java.util.Random;
62 import java.util.concurrent.BlockingQueue;
63 import java.util.concurrent.ExecutorService;
64 import java.util.concurrent.Executors;
65 import java.util.concurrent.LinkedBlockingQueue;
66 import java.security.PrivilegedExceptionAction;
67
68
69
70
71
72
73
74
75
76
77 public abstract class HBaseServer {
78
79
80
81
82 public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
83
84
85
86 public static final byte CURRENT_VERSION = 3;
87
88
89
90
91 private static final int MAX_QUEUE_SIZE_PER_HANDLER = 100;
92
93 public static final Log LOG =
94 LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer");
95
96 protected static final ThreadLocal<HBaseServer> SERVER =
97 new ThreadLocal<HBaseServer>();
98
99
100
101
102
103
104
105 public static HBaseServer get() {
106 return SERVER.get();
107 }
108
109
110
111
112 protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
113
114
115
116
117
118 public static InetAddress getRemoteIp() {
119 Call call = CurCall.get();
120 if (call != null) {
121 return call.connection.socket.getInetAddress();
122 }
123 return null;
124 }
125
126
127
128
129 public static String getRemoteAddress() {
130 InetAddress addr = getRemoteIp();
131 return (addr == null) ? null : addr.getHostAddress();
132 }
133
134 protected String bindAddress;
135 protected int port;
136 private int handlerCount;
137 private int priorityHandlerCount;
138 private int readThreads;
139 protected Class<? extends Writable> paramClass;
140 protected int maxIdleTime;
141
142
143 protected int thresholdIdleConnections;
144
145
146
147 int maxConnectionsToNuke;
148
149
150
151 protected HBaseRpcMetrics rpcMetrics;
152
153 protected Configuration conf;
154
155 @SuppressWarnings({"FieldCanBeLocal"})
156 private int maxQueueSize;
157 protected int socketSendBufferSize;
158 protected final boolean tcpNoDelay;
159 protected final boolean tcpKeepAlive;
160
161 volatile protected boolean running = true;
162 protected BlockingQueue<Call> callQueue;
163 protected BlockingQueue<Call> priorityCallQueue;
164
165 private int highPriorityLevel;
166
167 protected final List<Connection> connectionList =
168 Collections.synchronizedList(new LinkedList<Connection>());
169
170
171 private Listener listener = null;
172 protected Responder responder = null;
173 protected int numConnections = 0;
174 private Handler[] handlers = null;
175 private Handler[] priorityHandlers = null;
176 protected HBaseRPCErrorHandler errorHandler = null;
177
178
179
180
181
182
183
184
185
186
187
188 public static void bind(ServerSocket socket, InetSocketAddress address,
189 int backlog) throws IOException {
190 try {
191 socket.bind(address, backlog);
192 } catch (BindException e) {
193 BindException bindException =
194 new BindException("Problem binding to " + address + " : " +
195 e.getMessage());
196 bindException.initCause(e);
197 throw bindException;
198 } catch (SocketException e) {
199
200
201 if ("Unresolved address".equals(e.getMessage())) {
202 throw new UnknownHostException("Invalid hostname for server: " +
203 address.getHostName());
204 }
205 throw e;
206 }
207 }
208
209
210 private static class Call {
211 protected int id;
212 protected Writable param;
213 protected Connection connection;
214 protected long timestamp;
215
216 protected ByteBuffer response;
217
218 public Call(int id, Writable param, Connection connection) {
219 this.id = id;
220 this.param = param;
221 this.connection = connection;
222 this.timestamp = System.currentTimeMillis();
223 this.response = null;
224 }
225
226 @Override
227 public String toString() {
228 return param.toString() + " from " + connection.toString();
229 }
230
231 public void setResponse(ByteBuffer response) {
232 this.response = response;
233 }
234 }
235
236
237 private class Listener extends Thread {
238
239 private ServerSocketChannel acceptChannel = null;
240 private Selector selector = null;
241 private Reader[] readers = null;
242 private int currentReader = 0;
243 private InetSocketAddress address;
244 private Random rand = new Random();
245 private long lastCleanupRunTime = 0;
246
247 private long cleanupInterval = 10000;
248
249 private int backlogLength = conf.getInt("ipc.server.listen.queue.size", 128);
250
251 private ExecutorService readPool;
252
253 public Listener() throws IOException {
254 address = new InetSocketAddress(bindAddress, port);
255
256 acceptChannel = ServerSocketChannel.open();
257 acceptChannel.configureBlocking(false);
258
259
260 bind(acceptChannel.socket(), address, backlogLength);
261 port = acceptChannel.socket().getLocalPort();
262
263 selector= Selector.open();
264
265 readers = new Reader[readThreads];
266 readPool = Executors.newFixedThreadPool(readThreads);
267 for (int i = 0; i < readThreads; ++i) {
268 Selector readSelector = Selector.open();
269 Reader reader = new Reader(readSelector);
270 readers[i] = reader;
271 readPool.execute(reader);
272 }
273
274
275 acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
276 this.setName("IPC Server listener on " + port);
277 this.setDaemon(true);
278 }
279
280
281 private class Reader implements Runnable {
282 private volatile boolean adding = false;
283 private Selector readSelector = null;
284
285 Reader(Selector readSelector) {
286 this.readSelector = readSelector;
287 }
288 public void run() {
289 LOG.info("Starting SocketReader");
290 synchronized(this) {
291 while (running) {
292 SelectionKey key = null;
293 try {
294 readSelector.select();
295 while (adding) {
296 this.wait(1000);
297 }
298
299 Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
300 while (iter.hasNext()) {
301 key = iter.next();
302 iter.remove();
303 if (key.isValid()) {
304 if (key.isReadable()) {
305 doRead(key);
306 }
307 }
308 key = null;
309 }
310 } catch (InterruptedException e) {
311 if (running) {
312 LOG.info(getName() + "caught: " +
313 StringUtils.stringifyException(e));
314 }
315 } catch (IOException ex) {
316 LOG.error("Error in Reader", ex);
317 }
318 }
319 }
320 }
321
322
323
324
325
326
327
328
329 public void startAdd() {
330 adding = true;
331 readSelector.wakeup();
332 }
333
334 public synchronized SelectionKey registerChannel(SocketChannel channel)
335 throws IOException {
336 return channel.register(readSelector, SelectionKey.OP_READ);
337 }
338
339 public synchronized void finishAdd() {
340 adding = false;
341 this.notify();
342 }
343 }
344
345
346
347
348
349
350
351
352 private void cleanupConnections(boolean force) {
353 if (force || numConnections > thresholdIdleConnections) {
354 long currentTime = System.currentTimeMillis();
355 if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
356 return;
357 }
358 int start = 0;
359 int end = numConnections - 1;
360 if (!force) {
361 start = rand.nextInt() % numConnections;
362 end = rand.nextInt() % numConnections;
363 int temp;
364 if (end < start) {
365 temp = start;
366 start = end;
367 end = temp;
368 }
369 }
370 int i = start;
371 int numNuked = 0;
372 while (i <= end) {
373 Connection c;
374 synchronized (connectionList) {
375 try {
376 c = connectionList.get(i);
377 } catch (Exception e) {return;}
378 }
379 if (c.timedOut(currentTime)) {
380 if (LOG.isDebugEnabled())
381 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
382 closeConnection(c);
383 numNuked++;
384 end--;
385
386 c = null;
387 if (!force && numNuked == maxConnectionsToNuke) break;
388 }
389 else i++;
390 }
391 lastCleanupRunTime = System.currentTimeMillis();
392 }
393 }
394
395 @Override
396 public void run() {
397 LOG.info(getName() + ": starting");
398 SERVER.set(HBaseServer.this);
399
400 while (running) {
401 SelectionKey key = null;
402 try {
403 selector.select();
404 Iterator<SelectionKey> iter = selector.selectedKeys().iterator();
405 while (iter.hasNext()) {
406 key = iter.next();
407 iter.remove();
408 try {
409 if (key.isValid()) {
410 if (key.isAcceptable())
411 doAccept(key);
412 }
413 } catch (IOException ignored) {
414 }
415 key = null;
416 }
417 } catch (OutOfMemoryError e) {
418 if (errorHandler != null) {
419 if (errorHandler.checkOOME(e)) {
420 LOG.info(getName() + ": exiting on OOME");
421 closeCurrentConnection(key);
422 cleanupConnections(true);
423 return;
424 }
425 } else {
426
427
428
429 LOG.warn("Out of Memory in server select", e);
430 closeCurrentConnection(key);
431 cleanupConnections(true);
432 try { Thread.sleep(60000); } catch (Exception ignored) {}
433 }
434 } catch (Exception e) {
435 closeCurrentConnection(key);
436 }
437 cleanupConnections(false);
438 }
439 LOG.info("Stopping " + this.getName());
440
441 synchronized (this) {
442 try {
443 acceptChannel.close();
444 selector.close();
445 } catch (IOException ignored) { }
446
447 selector= null;
448 acceptChannel= null;
449
450
451 while (!connectionList.isEmpty()) {
452 closeConnection(connectionList.remove(0));
453 }
454 }
455 }
456
457 private void closeCurrentConnection(SelectionKey key) {
458 if (key != null) {
459 Connection c = (Connection)key.attachment();
460 if (c != null) {
461 if (LOG.isDebugEnabled())
462 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
463 closeConnection(c);
464 }
465 }
466 }
467
468 InetSocketAddress getAddress() {
469 return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
470 }
471
472 void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
473 Connection c;
474 ServerSocketChannel server = (ServerSocketChannel) key.channel();
475
476 SocketChannel channel;
477 while ((channel = server.accept()) != null) {
478 channel.configureBlocking(false);
479 channel.socket().setTcpNoDelay(tcpNoDelay);
480 channel.socket().setKeepAlive(tcpKeepAlive);
481
482 Reader reader = getReader();
483 try {
484 reader.startAdd();
485 SelectionKey readKey = reader.registerChannel(channel);
486 c = new Connection(channel, System.currentTimeMillis());
487 readKey.attach(c);
488 synchronized (connectionList) {
489 connectionList.add(numConnections, c);
490 numConnections++;
491 }
492 if (LOG.isDebugEnabled())
493 LOG.debug("Server connection from " + c.toString() +
494 "; # active connections: " + numConnections +
495 "; # queued calls: " + callQueue.size());
496 } finally {
497 reader.finishAdd();
498 }
499 }
500 }
501
502 void doRead(SelectionKey key) throws InterruptedException {
503 int count = 0;
504 Connection c = (Connection)key.attachment();
505 if (c == null) {
506 return;
507 }
508 c.setLastContact(System.currentTimeMillis());
509
510 try {
511 count = c.readAndProcess();
512 } catch (InterruptedException ieo) {
513 throw ieo;
514 } catch (Exception e) {
515 LOG.debug(getName() + ": readAndProcess threw exception " + e + ". Count of bytes read: " + count, e);
516 count = -1;
517 }
518 if (count < 0) {
519 if (LOG.isDebugEnabled())
520 LOG.debug(getName() + ": disconnecting client " +
521 c.getHostAddress() + ". Number of active connections: "+
522 numConnections);
523 closeConnection(c);
524
525 }
526 else {
527 c.setLastContact(System.currentTimeMillis());
528 }
529 }
530
531 synchronized void doStop() {
532 if (selector != null) {
533 selector.wakeup();
534 Thread.yield();
535 }
536 if (acceptChannel != null) {
537 try {
538 acceptChannel.socket().close();
539 } catch (IOException e) {
540 LOG.info(getName() + ":Exception in closing listener socket. " + e);
541 }
542 }
543 readPool.shutdownNow();
544 }
545
546
547
548 Reader getReader() {
549 currentReader = (currentReader + 1) % readers.length;
550 return readers[currentReader];
551 }
552 }
553
554
555 private class Responder extends Thread {
556 private Selector writeSelector;
557 private int pending;
558
559 final static int PURGE_INTERVAL = 900000;
560
561 Responder() throws IOException {
562 this.setName("IPC Server Responder");
563 this.setDaemon(true);
564 writeSelector = Selector.open();
565 pending = 0;
566 }
567
568 @Override
569 public void run() {
570 LOG.info(getName() + ": starting");
571 SERVER.set(HBaseServer.this);
572 long lastPurgeTime = 0;
573
574 while (running) {
575 try {
576 waitPending();
577 writeSelector.select(PURGE_INTERVAL);
578 Iterator<SelectionKey> iter = writeSelector.selectedKeys().iterator();
579 while (iter.hasNext()) {
580 SelectionKey key = iter.next();
581 iter.remove();
582 try {
583 if (key.isValid() && key.isWritable()) {
584 doAsyncWrite(key);
585 }
586 } catch (IOException e) {
587 LOG.info(getName() + ": doAsyncWrite threw exception " + e);
588 }
589 }
590 long now = System.currentTimeMillis();
591 if (now < lastPurgeTime + PURGE_INTERVAL) {
592 continue;
593 }
594 lastPurgeTime = now;
595
596
597
598
599 LOG.debug("Checking for old call responses.");
600 ArrayList<Call> calls;
601
602
603 synchronized (writeSelector.keys()) {
604 calls = new ArrayList<Call>(writeSelector.keys().size());
605 iter = writeSelector.keys().iterator();
606 while (iter.hasNext()) {
607 SelectionKey key = iter.next();
608 Call call = (Call)key.attachment();
609 if (call != null && key.channel() == call.connection.channel) {
610 calls.add(call);
611 }
612 }
613 }
614
615 for(Call call : calls) {
616 doPurge(call, now);
617 }
618 } catch (OutOfMemoryError e) {
619 if (errorHandler != null) {
620 if (errorHandler.checkOOME(e)) {
621 LOG.info(getName() + ": exiting on OOME");
622 return;
623 }
624 } else {
625
626
627
628
629
630 LOG.warn("Out of Memory in server select", e);
631 try { Thread.sleep(60000); } catch (Exception ignored) {}
632 }
633 } catch (Exception e) {
634 LOG.warn("Exception in Responder " +
635 StringUtils.stringifyException(e));
636 }
637 }
638 LOG.info("Stopping " + this.getName());
639 }
640
641 private void doAsyncWrite(SelectionKey key) throws IOException {
642 Call call = (Call)key.attachment();
643 if (call == null) {
644 return;
645 }
646 if (key.channel() != call.connection.channel) {
647 throw new IOException("doAsyncWrite: bad channel");
648 }
649
650 synchronized(call.connection.responseQueue) {
651 if (processResponse(call.connection.responseQueue, false)) {
652 try {
653 key.interestOps(0);
654 } catch (CancelledKeyException e) {
655
656
657
658
659
660 LOG.warn("Exception while changing ops : " + e);
661 }
662 }
663 }
664 }
665
666
667
668
669
670 private void doPurge(Call call, long now) {
671 synchronized (call.connection.responseQueue) {
672 Iterator<Call> iter = call.connection.responseQueue.listIterator(0);
673 while (iter.hasNext()) {
674 Call nextCall = iter.next();
675 if (now > nextCall.timestamp + PURGE_INTERVAL) {
676 closeConnection(nextCall.connection);
677 break;
678 }
679 }
680 }
681 }
682
683
684
685
686 @SuppressWarnings({"ConstantConditions"})
687 private boolean processResponse(final LinkedList<Call> responseQueue,
688 boolean inHandler) throws IOException {
689 boolean error = true;
690 boolean done = false;
691 int numElements;
692 Call call = null;
693 try {
694
695 synchronized (responseQueue) {
696
697
698
699 numElements = responseQueue.size();
700 if (numElements == 0) {
701 error = false;
702 return true;
703 }
704
705
706
707 call = responseQueue.removeFirst();
708 SocketChannel channel = call.connection.channel;
709 if (LOG.isDebugEnabled()) {
710 LOG.debug(getName() + ": responding to #" + call.id + " from " +
711 call.connection);
712 }
713
714
715
716 int numBytes = channelWrite(channel, call.response);
717 if (numBytes < 0) {
718 return true;
719 }
720 if (!call.response.hasRemaining()) {
721 call.connection.decRpcCount();
722
723 if (numElements == 1) {
724 done = true;
725 } else {
726 done = false;
727 }
728 if (LOG.isDebugEnabled()) {
729 LOG.debug(getName() + ": responding to #" + call.id + " from " +
730 call.connection + " Wrote " + numBytes + " bytes.");
731 }
732 } else {
733
734
735
736
737 call.connection.responseQueue.addFirst(call);
738
739 if (inHandler) {
740
741 call.timestamp = System.currentTimeMillis();
742
743 incPending();
744 try {
745
746
747 writeSelector.wakeup();
748 channel.register(writeSelector, SelectionKey.OP_WRITE, call);
749 } catch (ClosedChannelException e) {
750
751 done = true;
752 } finally {
753 decPending();
754 }
755 }
756 if (LOG.isDebugEnabled()) {
757 LOG.debug(getName() + ": responding to #" + call.id + " from " +
758 call.connection + " Wrote partial " + numBytes +
759 " bytes.");
760 }
761 }
762 error = false;
763 }
764 } finally {
765 if (error && call != null) {
766 LOG.warn(getName()+", call " + call + ": output error");
767 done = true;
768 closeConnection(call.connection);
769 }
770 }
771 return done;
772 }
773
774
775
776
777 void doRespond(Call call) throws IOException {
778 synchronized (call.connection.responseQueue) {
779 call.connection.responseQueue.addLast(call);
780 if (call.connection.responseQueue.size() == 1) {
781 processResponse(call.connection.responseQueue, true);
782 }
783 }
784 }
785
786 private synchronized void incPending() {
787 pending++;
788 }
789
790 private synchronized void decPending() {
791 pending--;
792 notify();
793 }
794
795 private synchronized void waitPending() throws InterruptedException {
796 while (pending > 0) {
797 wait();
798 }
799 }
800 }
801
802
803 private class Connection {
804 private boolean versionRead = false;
805
806 private boolean headerRead = false;
807
808 protected SocketChannel channel;
809 private ByteBuffer data;
810 private ByteBuffer dataLengthBuffer;
811 protected final LinkedList<Call> responseQueue;
812 private volatile int rpcCount = 0;
813 private long lastContact;
814 private int dataLength;
815 protected Socket socket;
816
817
818 private String hostAddress;
819 private int remotePort;
820 protected UserGroupInformation ticket = null;
821
822 public Connection(SocketChannel channel, long lastContact) {
823 this.channel = channel;
824 this.lastContact = lastContact;
825 this.data = null;
826 this.dataLengthBuffer = ByteBuffer.allocate(4);
827 this.socket = channel.socket();
828 InetAddress addr = socket.getInetAddress();
829 if (addr == null) {
830 this.hostAddress = "*Unknown*";
831 } else {
832 this.hostAddress = addr.getHostAddress();
833 }
834 this.remotePort = socket.getPort();
835 this.responseQueue = new LinkedList<Call>();
836 if (socketSendBufferSize != 0) {
837 try {
838 socket.setSendBufferSize(socketSendBufferSize);
839 } catch (IOException e) {
840 LOG.warn("Connection: unable to set socket send buffer size to " +
841 socketSendBufferSize);
842 }
843 }
844 }
845
846 @Override
847 public String toString() {
848 return getHostAddress() + ":" + remotePort;
849 }
850
851 public String getHostAddress() {
852 return hostAddress;
853 }
854
855 public void setLastContact(long lastContact) {
856 this.lastContact = lastContact;
857 }
858
859 public long getLastContact() {
860 return lastContact;
861 }
862
863
864 private boolean isIdle() {
865 return rpcCount == 0;
866 }
867
868
869 protected void decRpcCount() {
870 rpcCount--;
871 }
872
873
874 private void incRpcCount() {
875 rpcCount++;
876 }
877
878 protected boolean timedOut(long currentTime) {
879 return isIdle() && currentTime - lastContact > maxIdleTime;
880 }
881
882 public int readAndProcess() throws IOException, InterruptedException {
883 while (true) {
884
885
886
887 int count;
888 if (dataLengthBuffer.remaining() > 0) {
889 count = channelRead(channel, dataLengthBuffer);
890 if (count < 0 || dataLengthBuffer.remaining() > 0)
891 return count;
892 }
893
894 if (!versionRead) {
895
896 ByteBuffer versionBuffer = ByteBuffer.allocate(1);
897 count = channelRead(channel, versionBuffer);
898 if (count <= 0) {
899 return count;
900 }
901 int version = versionBuffer.get(0);
902
903 dataLengthBuffer.flip();
904 if (!HEADER.equals(dataLengthBuffer) || version != CURRENT_VERSION) {
905
906 LOG.warn("Incorrect header or version mismatch from " +
907 hostAddress + ":" + remotePort +
908 " got version " + version +
909 " expected version " + CURRENT_VERSION);
910 return -1;
911 }
912 dataLengthBuffer.clear();
913 versionRead = true;
914 continue;
915 }
916
917 if (data == null) {
918 dataLengthBuffer.flip();
919 dataLength = dataLengthBuffer.getInt();
920
921 if (dataLength == HBaseClient.PING_CALL_ID) {
922 dataLengthBuffer.clear();
923 return 0;
924 }
925 data = ByteBuffer.allocate(dataLength);
926 incRpcCount();
927 }
928
929 count = channelRead(channel, data);
930
931 if (data.remaining() == 0) {
932 dataLengthBuffer.clear();
933 data.flip();
934 if (headerRead) {
935 processData();
936 data = null;
937 return count;
938 }
939 processHeader();
940 headerRead = true;
941 data = null;
942 continue;
943 }
944 return count;
945 }
946 }
947
948
949 private void processHeader() throws IOException {
950
951
952
953 DataInputStream in =
954 new DataInputStream(new ByteArrayInputStream(data.array()));
955 String username = WritableUtils.readString(in);
956 ticket = UserGroupInformation.createRemoteUser(username);
957 }
958
959 private void processData() throws IOException, InterruptedException {
960 DataInputStream dis =
961 new DataInputStream(new ByteArrayInputStream(data.array()));
962 int id = dis.readInt();
963
964 if (LOG.isDebugEnabled())
965 LOG.debug(" got #" + id);
966
967 Writable param = ReflectionUtils.newInstance(paramClass, conf);
968 param.readFields(dis);
969
970 Call call = new Call(id, param, this);
971
972 if (priorityCallQueue != null && getQosLevel(param) > highPriorityLevel) {
973 priorityCallQueue.put(call);
974 } else {
975 callQueue.put(call);
976 }
977 }
978
979 protected synchronized void close() {
980 data = null;
981 dataLengthBuffer = null;
982 if (!channel.isOpen())
983 return;
984 try {socket.shutdownOutput();} catch(Exception ignored) {}
985 if (channel.isOpen()) {
986 try {channel.close();} catch(Exception ignored) {}
987 }
988 try {socket.close();} catch(Exception ignored) {}
989 }
990 }
991
992
993 private class Handler extends Thread {
994 private final BlockingQueue<Call> myCallQueue;
995 public Handler(final BlockingQueue<Call> cq, int instanceNumber) {
996 this.myCallQueue = cq;
997 this.setDaemon(true);
998
999 String threadName = "IPC Server handler " + instanceNumber + " on " + port;
1000 if (cq == priorityCallQueue) {
1001
1002 threadName = "PRI " + threadName;
1003 }
1004 this.setName(threadName);
1005 }
1006
1007 @Override
1008 public void run() {
1009 LOG.info(getName() + ": starting");
1010 SERVER.set(HBaseServer.this);
1011 final int buffersize = 16 * 1024;
1012 ByteArrayOutputStream buf = new ByteArrayOutputStream(buffersize);
1013 while (running) {
1014 try {
1015 final Call call = myCallQueue.take();
1016
1017 if (LOG.isDebugEnabled())
1018 LOG.debug(getName() + ": has #" + call.id + " from " +
1019 call.connection);
1020
1021 String errorClass = null;
1022 String error = null;
1023 Writable value = null;
1024
1025 CurCall.set(call);
1026 try {
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039 value = call(call.param, call.timestamp);
1040 } catch (Throwable e) {
1041 LOG.debug(getName()+", call "+call+": error: " + e, e);
1042 errorClass = e.getClass().getName();
1043 error = StringUtils.stringifyException(e);
1044 }
1045 CurCall.set(null);
1046
1047 if (buf.size() > buffersize) {
1048
1049
1050
1051 buf = new ByteArrayOutputStream(buffersize);
1052 } else {
1053 buf.reset();
1054 }
1055 DataOutputStream out = new DataOutputStream(buf);
1056 out.writeInt(call.id);
1057 out.writeBoolean(error != null);
1058
1059 if (error == null) {
1060 value.write(out);
1061 } else {
1062 WritableUtils.writeString(out, errorClass);
1063 WritableUtils.writeString(out, error);
1064 }
1065 call.setResponse(ByteBuffer.wrap(buf.toByteArray()));
1066 responder.doRespond(call);
1067 } catch (InterruptedException e) {
1068 if (running) {
1069 LOG.info(getName() + " caught: " +
1070 StringUtils.stringifyException(e));
1071 }
1072 } catch (OutOfMemoryError e) {
1073 if (errorHandler != null) {
1074 if (errorHandler.checkOOME(e)) {
1075 LOG.info(getName() + ": exiting on OOME");
1076 return;
1077 }
1078 } else {
1079
1080 throw e;
1081 }
1082 } catch (Exception e) {
1083 LOG.info(getName() + " caught: " +
1084 StringUtils.stringifyException(e));
1085 }
1086 }
1087 LOG.info(getName() + ": exiting");
1088 }
1089
1090 }
1091
1092
1093
1094
1095
1096
1097
1098
1099 private Function<Writable,Integer> qosFunction = null;
1100 public void setQosFunction(Function<Writable, Integer> newFunc) {
1101 qosFunction = newFunc;
1102 }
1103
1104 protected int getQosLevel(Writable param) {
1105 if (qosFunction == null) {
1106 return 0;
1107 }
1108
1109 Integer res = qosFunction.apply(param);
1110 if (res == null) {
1111 return 0;
1112 }
1113 return res;
1114 }
1115
1116
1117
1118
1119
1120
1121 protected HBaseServer(String bindAddress, int port,
1122 Class<? extends Writable> paramClass, int handlerCount,
1123 int priorityHandlerCount, Configuration conf, String serverName,
1124 int highPriorityLevel)
1125 throws IOException {
1126 this.bindAddress = bindAddress;
1127 this.conf = conf;
1128 this.port = port;
1129 this.paramClass = paramClass;
1130 this.handlerCount = handlerCount;
1131 this.priorityHandlerCount = priorityHandlerCount;
1132 this.socketSendBufferSize = 0;
1133 this.maxQueueSize = handlerCount * MAX_QUEUE_SIZE_PER_HANDLER;
1134 this.readThreads = conf.getInt(
1135 "ipc.server.read.threadpool.size",
1136 10);
1137 this.callQueue = new LinkedBlockingQueue<Call>(maxQueueSize);
1138 if (priorityHandlerCount > 0) {
1139 this.priorityCallQueue = new LinkedBlockingQueue<Call>(maxQueueSize);
1140 } else {
1141 this.priorityCallQueue = null;
1142 }
1143 this.highPriorityLevel = highPriorityLevel;
1144 this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
1145 this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
1146 this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
1147
1148
1149 listener = new Listener();
1150 this.port = listener.getAddress().getPort();
1151 this.rpcMetrics = new HBaseRpcMetrics(serverName,
1152 Integer.toString(this.port));
1153 this.tcpNoDelay = conf.getBoolean("ipc.server.tcpnodelay", false);
1154 this.tcpKeepAlive = conf.getBoolean("ipc.server.tcpkeepalive", true);
1155
1156
1157 responder = new Responder();
1158 }
1159
1160 protected void closeConnection(Connection connection) {
1161 synchronized (connectionList) {
1162 if (connectionList.remove(connection))
1163 numConnections--;
1164 }
1165 connection.close();
1166 }
1167
1168
1169
1170
1171 public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
1172
1173
1174 public synchronized void start() {
1175 responder.start();
1176 listener.start();
1177 handlers = new Handler[handlerCount];
1178
1179 for (int i = 0; i < handlerCount; i++) {
1180 handlers[i] = new Handler(callQueue, i);
1181 handlers[i].start();
1182 }
1183
1184 if (priorityHandlerCount > 0) {
1185 priorityHandlers = new Handler[priorityHandlerCount];
1186 for (int i = 0 ; i < priorityHandlerCount; i++) {
1187 priorityHandlers[i] = new Handler(priorityCallQueue, i);
1188 priorityHandlers[i].start();
1189 }
1190 }
1191 }
1192
1193
1194 public synchronized void stop() {
1195 LOG.info("Stopping server on " + port);
1196 running = false;
1197 if (handlers != null) {
1198 for (Handler handler : handlers) {
1199 if (handler != null) {
1200 handler.interrupt();
1201 }
1202 }
1203 }
1204 if (priorityHandlers != null) {
1205 for (Handler handler : priorityHandlers) {
1206 if (handler != null) {
1207 handler.interrupt();
1208 }
1209 }
1210 }
1211 listener.interrupt();
1212 listener.doStop();
1213 responder.interrupt();
1214 notifyAll();
1215 if (this.rpcMetrics != null) {
1216 this.rpcMetrics.shutdown();
1217 }
1218 }
1219
1220
1221
1222
1223
1224
1225 public synchronized void join() throws InterruptedException {
1226 while (running) {
1227 wait();
1228 }
1229 }
1230
1231
1232
1233
1234
1235 public synchronized InetSocketAddress getListenerAddress() {
1236 return listener.getAddress();
1237 }
1238
1239
1240
1241
1242
1243
1244
1245 public abstract Writable call(Writable param, long receiveTime)
1246 throws IOException;
1247
1248
1249
1250
1251
1252 public int getNumOpenConnections() {
1253 return numConnections;
1254 }
1255
1256
1257
1258
1259
1260 public int getCallQueueLen() {
1261 return callQueue.size();
1262 }
1263
1264
1265
1266
1267
1268 public void setErrorHandler(HBaseRPCErrorHandler handler) {
1269 this.errorHandler = handler;
1270 }
1271
1272
1273
1274
1275
1276
1277 private static int NIO_BUFFER_LIMIT = 8*1024;
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293 protected static int channelWrite(WritableByteChannel channel,
1294 ByteBuffer buffer) throws IOException {
1295 return (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1296 channel.write(buffer) : channelIO(null, channel, buffer);
1297 }
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311 protected static int channelRead(ReadableByteChannel channel,
1312 ByteBuffer buffer) throws IOException {
1313 return (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1314 channel.read(buffer) : channelIO(channel, null, buffer);
1315 }
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330 private static int channelIO(ReadableByteChannel readCh,
1331 WritableByteChannel writeCh,
1332 ByteBuffer buf) throws IOException {
1333
1334 int originalLimit = buf.limit();
1335 int initialRemaining = buf.remaining();
1336 int ret = 0;
1337
1338 while (buf.remaining() > 0) {
1339 try {
1340 int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
1341 buf.limit(buf.position() + ioSize);
1342
1343 ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
1344
1345 if (ret < ioSize) {
1346 break;
1347 }
1348
1349 } finally {
1350 buf.limit(originalLimit);
1351 }
1352 }
1353
1354 int nBytes = initialRemaining - buf.remaining();
1355 return (nBytes > 0) ? nBytes : ret;
1356 }
1357 }