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