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