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